From 57954bd9726ce9dfaae9b81d6886ddc4ffde8993 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 26 Sep 2023 15:34:25 -0400 Subject: [PATCH 01/64] Run arm suite against all validates_container tests (#28627) * Run arm suite against all validates_container tests * Exclude prebuildd container tests * Format * Add comment pointing to issue --- sdks/python/apache_beam/examples/wordcount_it_test.py | 9 +++++---- sdks/python/container/run_validatescontainer.sh | 10 ++++------ 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/examples/wordcount_it_test.py b/sdks/python/apache_beam/examples/wordcount_it_test.py index 0a5179e3bf03..4bd6e430420b 100644 --- a/sdks/python/apache_beam/examples/wordcount_it_test.py +++ b/sdks/python/apache_beam/examples/wordcount_it_test.py @@ -112,14 +112,15 @@ def test_wordcount_it_with_prebuilt_sdk_container_cloud_build(self): def test_wordcount_it_with_use_sibling_sdk_workers(self): self._run_wordcount_it(wordcount.run, experiment='use_sibling_sdk_workers') - @pytest.mark.it_dataflow_arm - def test_wordcount_fnapi_it_arm(self): - self._run_wordcount_it(wordcount.run, machine_type='t2a-standard-1') - def _run_wordcount_it(self, run_wordcount, **opts): test_pipeline = TestPipeline(is_integration_test=True) extra_opts = {} + if (test_pipeline.get_option('machine_type') == 't2a-standard-1' and + 'prebuild_sdk_container_engine' in opts): + # TODO(https://github.com/apache/beam/issues/28340) + pytest.skip('prebuild_sdk_container_engine not supported on ARM') + # Set extra options to the pipeline for test purpose test_output = '/'.join([ test_pipeline.get_option('output'), diff --git a/sdks/python/container/run_validatescontainer.sh b/sdks/python/container/run_validatescontainer.sh index f09712f74bc6..5ee3342a1efa 100755 --- a/sdks/python/container/run_validatescontainer.sh +++ b/sdks/python/container/run_validatescontainer.sh @@ -59,7 +59,7 @@ ARCH=${3:-"x86"} IMAGE_NAME="${IMAGE_PREFIX}python${PY_VERSION}_sdk" CONTAINER_PROJECT="sdks:python:container:py${PY_VERSION//.}" # Note: we substitute away the dot in the version. PY_INTERPRETER="python${PY_VERSION}" -TEST_SUITE_TAG="it_validatescontainer" +MACHINE_TYPE_ARGS="" XUNIT_FILE="pytest-$IMAGE_NAME.xml" @@ -88,12 +88,9 @@ if [[ "$ARCH" == "x86" ]]; then # Push the container gcloud docker -- push $CONTAINER:$TAG elif [[ "$ARCH" == "ARM" ]]; then - # Note: ARM test suites only run on github actions, where multi-arch Python SDK containers are already pushed during build. - # Reset the test suite tag to run ARM pipelines. - TEST_SUITE_TAG="it_dataflow_arm" - # Reset the multi-arch Python SDK container image tag. TAG=$MULTIARCH_TAG + MACHINE_TYPE_ARGS="--machine_type=t2a-standard-1" else printf "Please give a valid CPU architecture, either x86 or ARM." exit 1 @@ -126,7 +123,7 @@ SDK_LOCATION=$2 echo ">>> RUNNING DATAFLOW RUNNER VALIDATESCONTAINER TEST" pytest -o log_cli=True -o log_level=Info -o junit_suite_name=$IMAGE_NAME \ - -m=$TEST_SUITE_TAG \ + -m=it_validatescontainer \ --numprocesses=1 \ --timeout=1800 \ --junitxml=$XUNIT_FILE \ @@ -142,6 +139,7 @@ pytest -o log_cli=True -o log_level=Info -o junit_suite_name=$IMAGE_NAME \ --output=$GCS_LOCATION/output \ --sdk_location=$SDK_LOCATION \ --num_workers=1 \ + $MACHINE_TYPE_ARGS \ --docker_registry_push_url=$PREBUILD_SDK_CONTAINER_REGISTRY_PATH" echo ">>> SUCCESS DATAFLOW RUNNER VALIDATESCONTAINER TEST" From 7d78572baa21fcd5b235371161e1181168284a02 Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Tue, 26 Sep 2023 16:30:25 -0400 Subject: [PATCH 02/64] Add GoogleAdsIO to the list of built-in connectors (#28667) --- .../site/content/en/documentation/io/connectors.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/website/www/site/content/en/documentation/io/connectors.md b/website/www/site/content/en/documentation/io/connectors.md index b2179b36bc41..59b8898aa226 100644 --- a/website/www/site/content/en/documentation/io/connectors.md +++ b/website/www/site/content/en/documentation/io/connectors.md @@ -938,6 +938,20 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ ✘ + + GoogleAdsIO + ✔ + ✔ + + ✔ + native + + Not available + Not available + Not available + ✔ + ✔ + From 5c4bea2fdbb46e2055b26c5affe8362c0599423d Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Wed, 27 Sep 2023 11:11:48 -0400 Subject: [PATCH 03/64] add YAML SDK component to issue template (#28678) Signed-off-by: Jeffrey Kinard --- .github/ISSUE_TEMPLATE/bug.yml | 1 + .github/ISSUE_TEMPLATE/failing_test.yml | 1 + .github/ISSUE_TEMPLATE/feature.yml | 1 + .github/ISSUE_TEMPLATE/task.yml | 1 + .github/issue-rules.yml | 2 ++ 5 files changed, 6 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/bug.yml b/.github/ISSUE_TEMPLATE/bug.yml index 1ff96082c035..67f8b21445dc 100644 --- a/.github/ISSUE_TEMPLATE/bug.yml +++ b/.github/ISSUE_TEMPLATE/bug.yml @@ -63,6 +63,7 @@ body: - label: "Component: Go SDK" - label: "Component: Typescript SDK" - label: "Component: IO connector" + - label: "Component: Beam YAML" - label: "Component: Beam examples" - label: "Component: Beam playground" - label: "Component: Beam katas" diff --git a/.github/ISSUE_TEMPLATE/failing_test.yml b/.github/ISSUE_TEMPLATE/failing_test.yml index 0904719e6d2c..44e1cd720745 100644 --- a/.github/ISSUE_TEMPLATE/failing_test.yml +++ b/.github/ISSUE_TEMPLATE/failing_test.yml @@ -69,6 +69,7 @@ body: - label: "Component: Go SDK" - label: "Component: Typescript SDK" - label: "Component: IO connector" + - label: "Component: Beam YAML" - label: "Component: Beam examples" - label: "Component: Beam playground" - label: "Component: Beam katas" diff --git a/.github/ISSUE_TEMPLATE/feature.yml b/.github/ISSUE_TEMPLATE/feature.yml index deaa14287b8a..11234a5e1501 100644 --- a/.github/ISSUE_TEMPLATE/feature.yml +++ b/.github/ISSUE_TEMPLATE/feature.yml @@ -57,6 +57,7 @@ body: - label: "Component: Go SDK" - label: "Component: Typescript SDK" - label: "Component: IO connector" + - label: "Component: Beam YAML" - label: "Component: Beam examples" - label: "Component: Beam playground" - label: "Component: Beam katas" diff --git a/.github/ISSUE_TEMPLATE/task.yml b/.github/ISSUE_TEMPLATE/task.yml index 4fa1b241207d..477b91b181be 100644 --- a/.github/ISSUE_TEMPLATE/task.yml +++ b/.github/ISSUE_TEMPLATE/task.yml @@ -58,6 +58,7 @@ body: - label: "Component: Go SDK" - label: "Component: Typescript SDK" - label: "Component: IO connector" + - label: "Component: Beam YAML" - label: "Component: Beam examples" - label: "Component: Beam playground" - label: "Component: Beam katas" diff --git a/.github/issue-rules.yml b/.github/issue-rules.yml index c26cb84bf6ba..b01a22dafd78 100644 --- a/.github/issue-rules.yml +++ b/.github/issue-rules.yml @@ -36,6 +36,8 @@ rules: addLabels: ['typescript'] - contains: '[x] Component: IO' addLabels: ['io'] +- contains: '[x] Component: Beam YAML' + addLabels: ['yaml'] - contains: '[x] Component: Beam examples' addLabels: ['examples'] - contains: '[x] Component: Beam playground' From 725a2d62b27f657a0f5ab8d279508753de0908d1 Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Wed, 27 Sep 2023 11:14:06 -0400 Subject: [PATCH 04/64] Feature/add retry to gcp auth (#28475) * Update 2.50 release notes to include new Kafka topicPattern feature * Create groovy class for io performance tests Create gradle task and github actions config for GCS using this. * delete unnecessary class * fix env call * fix call to gradle * run on hosted runner for testing * add additional checkout * add destination for triggered tests * move env variables to correct location * try uploading against separate dataset * try without a user * update branch checkout, try to view the failure log * run on failure * update to use correct BigQuery instance * convert to matrix * add result reporting * add failure clause * remove failure clause, update to run on self-hosted * address comments, clean up build * clarify branching * Update auth to retry getting credentials from GCE * Re-order imports * Add test case * Update exception log * Add failure test * Update removal of retrying method * rework via mock * Clear credentials cache for idempotent tests * Remove handler after test Change retry timeout to facilitate shorter retrys for anonymous access cases * Change retry timeout to facilitate shorter retrys for anonymous access cases * reset credentials before and after test --- sdks/python/apache_beam/internal/gcp/auth.py | 14 +- .../apache_beam/internal/gcp/auth_test.py | 135 ++++++++++++++++++ 2 files changed, 146 insertions(+), 3 deletions(-) create mode 100644 sdks/python/apache_beam/internal/gcp/auth_test.py diff --git a/sdks/python/apache_beam/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py index 7e54ba0a4ba1..b2fda2c6e897 100644 --- a/sdks/python/apache_beam/internal/gcp/auth.py +++ b/sdks/python/apache_beam/internal/gcp/auth.py @@ -26,6 +26,7 @@ from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.utils import retry # google.auth is only available when Beam is installed with the gcp extra. try: @@ -152,8 +153,7 @@ def _get_service_credentials(pipeline_options): try: # pylint: disable=c-extension-no-member - credentials, _ = google.auth.default( - scopes=pipeline_options.view_as(GoogleCloudOptions).gcp_oauth_scopes) + credentials = _Credentials._get_credentials_with_retrys(pipeline_options) credentials = _Credentials._add_impersonation_credentials( credentials, pipeline_options) credentials = _ApitoolsCredentialsAdapter(credentials) @@ -164,10 +164,18 @@ def _get_service_credentials(pipeline_options): except Exception as e: _LOGGER.warning( 'Unable to find default credentials to use: %s\n' - 'Connecting anonymously.', + 'Connecting anonymously. This is expected if no ' + 'credentials are needed to access GCP resources.', e) return None + @staticmethod + @retry.with_exponential_backoff(num_retries=4, initial_delay_secs=2) + def _get_credentials_with_retrys(pipeline_options): + credentials, _ = google.auth.default( + scopes=pipeline_options.view_as(GoogleCloudOptions).gcp_oauth_scopes) + return credentials + @staticmethod def _add_impersonation_credentials(credentials, pipeline_options): gcs_options = pipeline_options.view_as(GoogleCloudOptions) diff --git a/sdks/python/apache_beam/internal/gcp/auth_test.py b/sdks/python/apache_beam/internal/gcp/auth_test.py new file mode 100644 index 000000000000..98fb828875b9 --- /dev/null +++ b/sdks/python/apache_beam/internal/gcp/auth_test.py @@ -0,0 +1,135 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import logging +import unittest + +import mock + +from apache_beam.internal.gcp import auth +from apache_beam.options.pipeline_options import GoogleCloudOptions +from apache_beam.options.pipeline_options import PipelineOptions + +try: + import google.auth as gauth +except ImportError: + gauth = None + + +class MockLoggingHandler(logging.Handler): + """Mock logging handler to check for expected logs.""" + def __init__(self, *args, **kwargs): + self.reset() + logging.Handler.__init__(self, *args, **kwargs) + + def emit(self, record): + self.messages[record.levelname.lower()].append(record.getMessage()) + + def reset(self): + self.messages = { + 'debug': [], + 'info': [], + 'warning': [], + 'error': [], + 'critical': [], + } + + +@unittest.skipIf(gauth is None, 'Google Auth dependencies are not installed') +class AuthTest(unittest.TestCase): + @mock.patch('google.auth.default') + def test_auth_with_retrys(self, unused_mock_arg): + pipeline_options = PipelineOptions() + pipeline_options.view_as( + GoogleCloudOptions).impersonate_service_account = False + + credentials = ('creds', 1) + + self.is_called = False + + def side_effect(scopes=None): + if self.is_called: + return credentials + else: + self.is_called = True + raise IOError('Failed') + + google_auth_mock = mock.MagicMock() + gauth.default = google_auth_mock + google_auth_mock.side_effect = side_effect + + # _Credentials caches the actual credentials. + # This resets it for idempotent tests. + if auth._Credentials._credentials_init: + auth._Credentials._credentials_init = False + auth._Credentials._credentials = None + + returned_credentials = auth.get_service_credentials(pipeline_options) + + # _Credentials caches the actual credentials. + # This resets it for idempotent tests. + if auth._Credentials._credentials_init: + auth._Credentials._credentials_init = False + auth._Credentials._credentials = None + + self.assertEqual('creds', returned_credentials._google_auth_credentials) + + @mock.patch( + 'apache_beam.internal.gcp.auth._Credentials._get_credentials_with_retrys') + def test_auth_with_retrys_always_fail(self, unused_mock_arg): + pipeline_options = PipelineOptions() + pipeline_options.view_as( + GoogleCloudOptions).impersonate_service_account = False + + loggerHandler = MockLoggingHandler() + + auth._LOGGER.addHandler(loggerHandler) + + #Remove call to retrying method, as otherwise test takes ~10 minutes to run + def raise_(scopes=None): + raise IOError('Failed') + + retry_auth_mock = mock.MagicMock() + auth._Credentials._get_credentials_with_retrys = retry_auth_mock + retry_auth_mock.side_effect = raise_ + + # _Credentials caches the actual credentials. + # This resets it for idempotent tests. + if auth._Credentials._credentials_init: + auth._Credentials._credentials_init = False + auth._Credentials._credentials = None + + returned_credentials = auth.get_service_credentials(pipeline_options) + + self.assertEqual(None, returned_credentials) + self.assertEqual([ + 'Unable to find default credentials to use: Failed\n' + 'Connecting anonymously. This is expected if no credentials are ' + 'needed to access GCP resources.' + ], + loggerHandler.messages.get('warning')) + + # _Credentials caches the actual credentials. + # This resets it for idempotent tests. + if auth._Credentials._credentials_init: + auth._Credentials._credentials_init = False + auth._Credentials._credentials = None + + auth._LOGGER.removeHandler(loggerHandler) + + +if __name__ == '__main__': + unittest.main() From f35a4130ef35b534d93e2dfa9f88f22cf8cd1d27 Mon Sep 17 00:00:00 2001 From: ddebowczyk92 Date: Wed, 27 Sep 2023 17:14:59 +0200 Subject: [PATCH 05/64] Make SerializableConfiguration cacheable (#28590) --- .../io/hadoop/SerializableConfiguration.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java index bb59b07cc280..40099e634678 100644 --- a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java +++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.hadoop; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; @@ -39,6 +41,8 @@ public class SerializableConfiguration implements Externalizable { private transient Configuration conf; + private transient byte[] serializationCache; + public SerializableConfiguration() {} public SerializableConfiguration(Configuration conf) { @@ -49,17 +53,30 @@ public SerializableConfiguration(Configuration conf) { } public Configuration get() { + if (serializationCache != null) { + serializationCache = null; + } return conf; } @Override public void writeExternal(ObjectOutput out) throws IOException { + if (serializationCache == null) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(512); + try (DataOutputStream dos = new DataOutputStream(baos)) { + conf.write(dos); + serializationCache = baos.toByteArray(); + } + } out.writeUTF(conf.getClass().getCanonicalName()); - conf.write(out); + out.write(serializationCache); } @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + if (serializationCache != null) { + serializationCache = null; + } String className = in.readUTF(); try { conf = From ced3de3feaed9c5c2b201576ab4893091c466ea5 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Wed, 27 Sep 2023 13:14:29 -0700 Subject: [PATCH 06/64] Require --sdk_location for Dataflow pipelines running with dev SDKs. (#28670) --- .../runners/dataflow/dataflow_runner.py | 10 ++++++++++ .../runners/dataflow/dataflow_runner_test.py | 17 ++++++++++++----- .../apache_beam/runners/portability/stager.py | 8 +++----- 3 files changed, 25 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 3d612bd6ec0f..950bff768ef4 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -586,6 +586,16 @@ def _check_and_add_missing_options(options): elif debug_options.lookup_experiment('enable_prime'): dataflow_service_options.append('enable_prime') + sdk_location = options.view_as(SetupOptions).sdk_location + if 'dev' in beam.version.__version__ and sdk_location == 'default': + raise ValueError( + "When launching Dataflow Jobs with an unreleased SDK, " + "please provide an SDK distribution in the --sdk_location option " + "to use consistent SDK version at " + "pipeline submission and runtime. To ignore this error and use the " + "SDK installed in Dataflow dev containers, use " + "--sdk_location=container.") + # Streaming only supports using runner v2 (aka unified worker). # Runner v2 only supports using streaming engine (aka windmill service) if options.view_as(StandardOptions).streaming: diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index b58531acc6a9..6258aa8f213b 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -501,7 +501,7 @@ def expand(self, pcoll): self.assertIn(packed_step_name, transform_names) def test_batch_is_runner_v2(self): - options = PipelineOptions() + options = PipelineOptions(['--sdk_location=container']) _check_and_add_missing_options(options) for expected in ['beam_fn_api', 'use_unified_worker', @@ -512,7 +512,7 @@ def test_batch_is_runner_v2(self): expected) def test_streaming_is_runner_v2(self): - options = PipelineOptions(['--streaming']) + options = PipelineOptions(['--sdk_location=container', '--streaming']) _check_and_add_missing_options(options) for expected in ['beam_fn_api', 'use_unified_worker', @@ -525,7 +525,11 @@ def test_streaming_is_runner_v2(self): expected) def test_dataflow_service_options_enable_prime_sets_runner_v2(self): - options = PipelineOptions(['--dataflow_service_options=enable_prime']) + options = PipelineOptions([ + '--sdk_location=container', + '--streaming', + '--dataflow_service_options=enable_prime' + ]) _check_and_add_missing_options(options) for expected in ['beam_fn_api', 'use_unified_worker', @@ -535,8 +539,11 @@ def test_dataflow_service_options_enable_prime_sets_runner_v2(self): options.view_as(DebugOptions).lookup_experiment(expected, False), expected) - options = PipelineOptions( - ['--streaming', '--dataflow_service_options=enable_prime']) + options = PipelineOptions([ + '--sdk_location=container', + '--streaming', + '--dataflow_service_options=enable_prime' + ]) _check_and_add_missing_options(options) for expected in ['beam_fn_api', 'use_unified_worker', diff --git a/sdks/python/apache_beam/runners/portability/stager.py b/sdks/python/apache_beam/runners/portability/stager.py index 1f093b1d7bc3..ace573de0a62 100644 --- a/sdks/python/apache_beam/runners/portability/stager.py +++ b/sdks/python/apache_beam/runners/portability/stager.py @@ -594,11 +594,9 @@ def _create_extra_packages(extra_packages, temp_dir): '".tar", ".tar.gz", ".whl" or ".zip" instead of %s' % package) if os.path.basename(package).endswith('.whl'): _LOGGER.warning( - 'The .whl package "%s" is provided in --extra_package. ' - 'This functionality is not officially supported. Since wheel ' - 'packages are binary distributions, this package must be ' - 'binary-compatible with the worker environment (e.g. Python 2.7 ' - 'running on an x64 Linux host).' % package) + 'The .whl package "%s" provided in --extra_package ' + 'must be binary-compatible with the worker runtime environment.' % + package) if not os.path.isfile(package): if Stager._is_remote_path(package): From 1c50fd2e6a8bd43f767f2ef49d4d5ecbf9c84f68 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Wed, 27 Sep 2023 14:44:41 -0700 Subject: [PATCH 07/64] Upgrade transforms without upgrading the pipelines (#28210) * Upgrade Java transforms without upgrading the pipelines * Addresses reviewer comments * Reduce visibility of the test-only constructor * Fix compile errors * Fix spotless * Addressing reviewer comments * Do not bundle Transform Service Launcher in the harness * Fix harness build and a fix for when a runner invokes toProto() multiple times --- runners/core-construction-java/build.gradle | 1 + .../core/construction/CombineTranslation.java | 6 +- .../CreatePCollectionViewTranslation.java | 2 +- .../runners/core/construction/External.java | 10 +- .../ExternalTranslationOptions.java | 43 ++ .../ExternalTranslationOptionsRegistrar.java | 36 ++ .../core/construction/FlattenTranslator.java | 2 +- .../construction/GroupByKeyTranslation.java | 2 +- .../GroupIntoBatchesTranslation.java | 4 +- .../core/construction/ImpulseTranslation.java | 2 +- .../construction/PTransformTranslation.java | 102 ++++- .../construction/PipelineTranslation.java | 15 + .../core/construction/ReadTranslation.java | 4 +- .../construction/ReshuffleTranslation.java | 2 +- .../core/construction/SplittableParDo.java | 2 +- .../construction/TestStreamTranslation.java | 2 +- .../core/construction/TransformUpgrader.java | 330 ++++++++++++++++ .../construction/WindowIntoTranslation.java | 5 +- .../construction/WriteFilesTranslation.java | 2 +- .../construction/TransformUpgraderTest.java | 369 ++++++++++++++++++ .../FlinkStreamingTransformTranslators.java | 2 +- .../beam/runners/dataflow/DataflowRunner.java | 6 + .../dataflow/PrimitiveParDoSingleFactory.java | 2 +- .../runners/dataflow/DataflowRunnerTest.java | 2 +- .../samza/translation/SamzaPublishView.java | 2 +- .../StreamingTransformTranslator.java | 4 +- .../expansion/service/ExpansionService.java | 65 +++ sdks/java/harness/build.gradle | 2 + .../gcp/pubsub/PubSubPayloadTranslation.java | 9 +- 29 files changed, 994 insertions(+), 41 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle index feac7c37c8e0..f593865b3fe9 100644 --- a/runners/core-construction-java/build.gradle +++ b/runners/core-construction-java/build.gradle @@ -55,6 +55,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:extensions:avro") implementation project(path: ":sdks:java:fn-execution") + implementation project(path: ":sdks:java:transform-service:launcher") implementation library.java.vendored_grpc_1_54_0 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.classgraph diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index 3f902acf250c..fbe4876b414f 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -61,7 +61,7 @@ public static class CombinePerKeyPayloadTranslator private CombinePerKeyPayloadTranslator() {} @Override - public String getUrn(Combine.PerKey transform) { + public String getUrn() { return COMBINE_PER_KEY_TRANSFORM_URN; } @@ -108,7 +108,7 @@ public static class CombineGloballyPayloadTranslator private CombineGloballyPayloadTranslator() {} @Override - public String getUrn(Combine.Globally transform) { + public String getUrn() { return COMBINE_GLOBALLY_TRANSFORM_URN; } @@ -165,7 +165,7 @@ public static class CombineGroupedValuesPayloadTranslator private CombineGroupedValuesPayloadTranslator() {} @Override - public String getUrn(Combine.GroupedValues transform) { + public String getUrn() { return COMBINE_GROUPED_VALUES_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java index a679737fd616..71038564ec4c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java @@ -90,7 +90,7 @@ public static PCollectionView getView( static class CreatePCollectionViewTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(View.CreatePCollectionView transform) { + public String getUrn() { return PTransformTranslation.CREATE_VIEW_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java index cedd8875751f..534a2b5fe0e6 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java @@ -295,7 +295,7 @@ public OutputT expand(InputT input) { response .getComponents() .toBuilder() - .putAllEnvironments(resolveArtifacts(newEnvironmentsWithDependencies)) + .putAllEnvironments(resolveArtifacts(newEnvironmentsWithDependencies, endpoint)) .build(); expandedTransform = response.getTransform(); expandedRequirements = response.getRequirementsList(); @@ -338,8 +338,8 @@ public OutputT expand(InputT input) { return toOutputCollection(outputMapBuilder.build()); } - private Map resolveArtifacts( - Map environments) { + static Map resolveArtifacts( + Map environments, Endpoints.ApiServiceDescriptor endpoint) { if (environments.size() == 0) { return environments; } @@ -367,7 +367,7 @@ private Map resolveArtifacts( } } - private RunnerApi.Environment resolveArtifacts( + private static RunnerApi.Environment resolveArtifacts( ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceBlockingStub retrievalStub, RunnerApi.Environment environment) throws IOException { @@ -378,7 +378,7 @@ private RunnerApi.Environment resolveArtifacts( .build(); } - private List resolveArtifacts( + private static List resolveArtifacts( ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceBlockingStub retrievalStub, List artifacts) throws IOException { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java new file mode 100644 index 000000000000..4b3ef24ca1d2 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core.construction; + +import java.util.List; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.resourcehints.ResourceHintsOptions.EmptyListDefault; + +public interface ExternalTranslationOptions extends PipelineOptions { + + @Description("Set of URNs of transforms to be overriden using the transform service.") + @Default.InstanceFactory(EmptyListDefault.class) + List getTransformsToOverride(); + + void setTransformsToOverride(List transformsToOverride); + + @Description("Address of an already available transform service.") + String getTransformServiceAddress(); + + void setTransformServiceAddress(String transformServiceAddress); + + @Description("An available Beam version which will be used to start a transform service.") + String getTransformServiceBeamVersion(); + + void setTransformServiceBeamVersion(String transformServiceBeamVersion); +} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java new file mode 100644 index 000000000000..6296f4c83775 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core.construction; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; + +/** A registrar for ExternalTranslationOptions. */ +@AutoService(PipelineOptionsRegistrar.class) +@Internal +public class ExternalTranslationOptionsRegistrar implements PipelineOptionsRegistrar { + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>builder() + .add(ExternalTranslationOptions.class) + .build(); + } +} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java index 37c09663c5a2..201a65e6233c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java @@ -43,7 +43,7 @@ public static TransformPayloadTranslator create() { private FlattenTranslator() {} @Override - public String getUrn(Flatten.PCollections transform) { + public String getUrn() { return PTransformTranslation.FLATTEN_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java index e6bbbf0767a5..183fa7ffcdc9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java @@ -38,7 +38,7 @@ public class GroupByKeyTranslation { static class GroupByKeyTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(GroupByKey transform) { + public String getUrn() { return PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java index c91e9cedb9ac..7c81afd8ae07 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java @@ -39,7 +39,7 @@ public class GroupIntoBatchesTranslation { static class GroupIntoBatchesTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(GroupIntoBatches transform) { + public String getUrn() { return PTransformTranslation.GROUP_INTO_BATCHES_URN; } @@ -61,7 +61,7 @@ public RunnerApi.FunctionSpec translate( static class ShardedGroupIntoBatchesTranslator implements TransformPayloadTranslator.WithShardedKey> { @Override - public String getUrn(GroupIntoBatches.WithShardedKey transform) { + public String getUrn() { return PTransformTranslation.GROUP_INTO_BATCHES_WITH_SHARDED_KEY_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java index 3de0ce9de8ac..25f0cd7749b5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java @@ -37,7 +37,7 @@ public class ImpulseTranslation { private static class ImpulseTranslator implements TransformPayloadTranslator { @Override - public String getUrn(Impulse transform) { + public String getUrn() { return PTransformTranslation.IMPULSE_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 2acd77885fcc..8f415e718e95 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -37,15 +37,20 @@ import org.apache.beam.runners.core.construction.ExternalTranslation.ExternalTranslator; import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoTranslator; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.common.ReflectHelpers.ObjectsClassComparator; 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.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; 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; @@ -54,6 +59,8 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Utilities for converting {@link PTransform PTransforms} to {@link RunnerApi Runner API protocol @@ -65,10 +72,14 @@ "keyfor" }) // TODO(https://github.com/apache/beam/issues/20497) public class PTransformTranslation { + + private static final Logger LOG = LoggerFactory.getLogger(PTransformTranslation.class); + // We specifically copy the values here so that they can be used in switch case statements // and we validate that the value matches the actual URN in the static block below. // Primitives + public static final String CREATE_TRANSFORM_URN = "beam:transform:create:v1"; public static final String PAR_DO_TRANSFORM_URN = "beam:transform:pardo:v1"; public static final String FLATTEN_TRANSFORM_URN = "beam:transform:flatten:v1"; public static final String GROUP_BY_KEY_TRANSFORM_URN = "beam:transform:group_by_key:v1"; @@ -83,6 +94,10 @@ public class PTransformTranslation { public static final ImmutableSet RUNNER_IMPLEMENTED_TRANSFORMS = ImmutableSet.of(GROUP_BY_KEY_TRANSFORM_URN, IMPULSE_TRANSFORM_URN); + public static final String CONFIG_ROW_KEY = "config_row"; + + public static final String CONFIG_ROW_SCHEMA_KEY = "config_row_schema"; + // DeprecatedPrimitives /** * @deprecated SDKs should move away from creating `Read` transforms and migrate to using Impulse @@ -435,10 +450,9 @@ public RunnerApi.PTransform translate( RunnerApi.PTransform.Builder transformBuilder = translateAppliedPTransform(appliedPTransform, subtransforms, components); - FunctionSpec spec = - KNOWN_PAYLOAD_TRANSLATORS - .get(appliedPTransform.getTransform().getClass()) - .translate(appliedPTransform, components); + TransformPayloadTranslator payloadTranslator = + KNOWN_PAYLOAD_TRANSLATORS.get(appliedPTransform.getTransform().getClass()); + FunctionSpec spec = payloadTranslator.translate(appliedPTransform, components); if (spec != null) { transformBuilder.setSpec(spec); @@ -461,6 +475,33 @@ public RunnerApi.PTransform translate( } } } + + Row configRow = null; + try { + configRow = payloadTranslator.toConfigRow(appliedPTransform.getTransform()); + } catch (UnsupportedOperationException e) { + // Optional toConfigRow() has not been implemented. We can just ignore. + } catch (Exception e) { + LOG.warn( + "Could not attach the config row for transform " + + appliedPTransform.getTransform().getName() + + ": " + + e); + // Ignoring the error and continuing with the translation since attaching config rows is + // optional. + } + if (configRow != null) { + transformBuilder.putAnnotations( + CONFIG_ROW_KEY, + ByteString.copyFrom( + CoderUtils.encodeToByteArray(RowCoder.of(configRow.getSchema()), configRow))); + + transformBuilder.putAnnotations( + CONFIG_ROW_SCHEMA_KEY, + ByteString.copyFrom( + SchemaTranslation.schemaToProto(configRow.getSchema(), true).toByteArray())); + } + return transformBuilder.build(); } } @@ -508,14 +549,63 @@ static RunnerApi.PTransform.Builder translateAppliedPTransform( * *

When going to a protocol buffer message, the translator produces a payload corresponding to * the Java representation while registering components that payload references. + * + *

Also, provides methods for generating a Row-based constructor config for the transform that + * can be later used to re-construct the transform. */ public interface TransformPayloadTranslator> { - String getUrn(T transform); + /** + * Provides a unique URN for transforms represented by this {@code TransformPayloadTranslator}. + */ + String getUrn(); + + /** + * Same as {@link #getUrn()} but the returned URN may depend on the transform provided. + * + *

Only override this if the same {@code TransformPayloadTranslator} used for multiple + * transforms. Otherwise, use {@link #getUrn()}. + */ + default String getUrn(T transform) { + return getUrn(); + } + + /** */ + /** + * Translates the given transform represented by the provided {@code AppliedPTransform} to a + * {@code FunctionSpec} with a URN and a payload. + * + * @param application an {@code AppliedPTransform} that includes the transform to be expanded. + * @param components components of the pipeline that includes the transform. + * @return a generated spec for the transform to be included in the pipeline proto. If return + * value is null, transform should include an empty spec. + * @throws IOException + */ @Nullable FunctionSpec translate(AppliedPTransform application, SdkComponents components) throws IOException; + /** + * Generates a Row-based construction configuration for the provided transform. + * + * @param transform a transform represented by the current {@code TransformPayloadTranslator}. + * @return + */ + default Row toConfigRow(T transform) { + throw new UnsupportedOperationException("Not implemented"); + } + + /** + * Construts a transform from a provided Row-based construction configuration. + * + * @param configRow a construction configuration similar to what would be generated by the + * {@link #toConfigRow(PTransform)} method. + * @return a transform represented by the current {@code TransformPayloadTranslator}. + */ + default T fromConfigRow(Row configRow) { + throw new UnsupportedOperationException("Not implemented"); + } + /** * A {@link TransformPayloadTranslator} for transforms that contain no references to components, * so they do not need a specialized rehydration. @@ -526,7 +616,7 @@ abstract class NotSerializable> public static NotSerializable forUrn(final String urn) { return new NotSerializable>() { @Override - public String getUrn(PTransform transform) { + public String getUrn() { return urn; } }; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java index 53553e7062b3..e39a38a74c2c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java @@ -102,6 +102,21 @@ public void visitPrimitiveTransform(Node node) { // TODO(JIRA-5649): Don't even emit these transforms in the generated protos. res = elideDeprecatedViews(res); } + + ExternalTranslationOptions externalTranslationOptions = + pipeline.getOptions().as(ExternalTranslationOptions.class); + List urnsToOverride = externalTranslationOptions.getTransformsToOverride(); + if (urnsToOverride.size() > 0) { + try (TransformUpgrader upgrader = TransformUpgrader.of()) { + res = + upgrader.upgradeTransformsViaTransformService( + res, urnsToOverride, externalTranslationOptions); + } catch (Exception e) { + throw new RuntimeException( + "Could not override the transforms with URNs " + urnsToOverride, e); + } + } + // Validate that translation didn't produce an invalid pipeline. PipelineValidator.validate(res); return res; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index 40a7205b5d57..f04d25509593 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -154,7 +154,7 @@ public static TransformPayloadTranslator create() { private UnboundedReadPayloadTranslator() {} @Override - public String getUrn(SplittableParDo.PrimitiveUnboundedRead transform) { + public String getUrn() { return PTransformTranslation.READ_TRANSFORM_URN; } @@ -181,7 +181,7 @@ public static TransformPayloadTranslator create() { private BoundedReadPayloadTranslator() {} @Override - public String getUrn(SplittableParDo.PrimitiveBoundedRead transform) { + public String getUrn() { return PTransformTranslation.READ_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java index 98d39c8ff0ac..bd91673f3818 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java @@ -38,7 +38,7 @@ public class ReshuffleTranslation { static class ReshuffleTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(Reshuffle transform) { + public String getUrn() { return PTransformTranslation.RESHUFFLE_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index 42c9e523e965..5ea2c4968dd9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -395,7 +395,7 @@ public static TransformPayloadTranslator create() { private ProcessKeyedElementsTranslator() {} @Override - public String getUrn(ProcessKeyedElements transform) { + public String getUrn() { return PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java index aa582bf14f3c..53bb324d03fa 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java @@ -168,7 +168,7 @@ static TestStream.Event eventFromProto( /** A translator registered to translate {@link TestStream} objects to protobuf representation. */ static class TestStreamTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(TestStream transform) { + public String getUrn() { return TEST_STREAM_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java new file mode 100644 index 000000000000..d657bb31b184 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -0,0 +1,330 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core.construction; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import org.apache.beam.model.expansion.v1.ExpansionApi; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; +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.Strings; + +/** + * A utility class that allows upgrading transforms of a given pipeline using the Beam Transform + * Service. + */ +public class TransformUpgrader implements AutoCloseable { + private static final String UPGRADE_NAMESPACE = "transform:upgrade:"; + + private ExpansionServiceClientFactory clientFactory; + + private TransformUpgrader() { + // Creating a default 'ExpansionServiceClientFactory' instance per 'TransformUpgrader' instance + // so that each instance can maintain a set of live channels and close them independently. + clientFactory = + DefaultExpansionServiceClientFactory.create( + endPoint -> ManagedChannelBuilder.forTarget(endPoint.getUrl()).usePlaintext().build()); + } + + private TransformUpgrader(ExpansionServiceClientFactory clientFactory) { + this.clientFactory = clientFactory; + } + + public static TransformUpgrader of() { + return new TransformUpgrader(); + } + + @VisibleForTesting + static TransformUpgrader of(ExpansionServiceClientFactory clientFactory) { + return new TransformUpgrader(clientFactory); + } + + /** + * Upgrade identified transforms in a given pipeline using the Transform Service. + * + * @param pipeline the pipeline proto. + * @param urnsToOverride URNs of the transforms to be overridden. + * @param options options for determining the transform service to use. + * @return pipelines with transforms upgraded using the Transform Service. + * @throws Exception + */ + public RunnerApi.Pipeline upgradeTransformsViaTransformService( + RunnerApi.Pipeline pipeline, List urnsToOverride, ExternalTranslationOptions options) + throws IOException, TimeoutException { + List transformsToOverride = + pipeline.getComponents().getTransformsMap().entrySet().stream() + .filter( + entry -> { + String urn = entry.getValue().getSpec().getUrn(); + if (urn != null && urnsToOverride.contains(urn)) { + return true; + } + return false; + }) + .map( + entry -> { + return entry.getKey(); + }) + .collect(Collectors.toList()); + + String serviceAddress; + TransformServiceLauncher service = null; + + if (options.getTransformServiceAddress() != null) { + serviceAddress = options.getTransformServiceAddress(); + } else if (options.getTransformServiceBeamVersion() != null) { + String projectName = UUID.randomUUID().toString(); + int port = findAvailablePort(); + service = TransformServiceLauncher.forProject(projectName, port); + service.setBeamVersion(options.getTransformServiceBeamVersion()); + + // Starting the transform service. + service.start(); + service.waitTillUp(-1); + serviceAddress = "localhost:" + Integer.toString(port); + } else { + throw new IllegalArgumentException( + "Either option TransformServiceAddress or option TransformServiceBeamVersion should be " + + "provided to override a transform using the transform service"); + } + + Endpoints.ApiServiceDescriptor expansionServiceEndpoint = + Endpoints.ApiServiceDescriptor.newBuilder().setUrl(serviceAddress).build(); + + for (String transformId : transformsToOverride) { + pipeline = + updateTransformViaTransformService(pipeline, transformId, expansionServiceEndpoint); + } + + if (service != null) { + service.shutdown(); + } + + return pipeline; + } + + private < + InputT extends PInput, + OutputT extends POutput, + TransformT extends org.apache.beam.sdk.transforms.PTransform> + RunnerApi.Pipeline updateTransformViaTransformService( + RunnerApi.Pipeline runnerAPIpipeline, + String transformId, + Endpoints.ApiServiceDescriptor transformServiceEndpoint) + throws IOException { + PTransform transformToUpgrade = + runnerAPIpipeline.getComponents().getTransformsMap().get(transformId); + if (transformToUpgrade == null) { + throw new IllegalArgumentException("Could not find a transform with the ID " + transformId); + } + + ByteString configRowBytes = + transformToUpgrade.getAnnotationsOrThrow(PTransformTranslation.CONFIG_ROW_KEY); + ByteString configRowSchemaBytes = + transformToUpgrade.getAnnotationsOrThrow(PTransformTranslation.CONFIG_ROW_SCHEMA_KEY); + SchemaApi.Schema configRowSchemaProto = + SchemaApi.Schema.parseFrom(configRowSchemaBytes.toByteArray()); + + ExternalTransforms.ExternalConfigurationPayload payload = + ExternalTransforms.ExternalConfigurationPayload.newBuilder() + .setSchema(configRowSchemaProto) + .setPayload(configRowBytes) + .build(); + + RunnerApi.PTransform.Builder ptransformBuilder = + RunnerApi.PTransform.newBuilder() + .setUniqueName(transformToUpgrade.getUniqueName() + "_external") + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn(transformToUpgrade.getSpec().getUrn()) + .setPayload(ByteString.copyFrom(payload.toByteArray())) + .build()); + + for (Map.Entry entry : transformToUpgrade.getInputsMap().entrySet()) { + ptransformBuilder.putInputs(entry.getKey(), entry.getValue()); + } + for (Map.Entry entry : transformToUpgrade.getOutputsMap().entrySet()) { + ptransformBuilder.putOutputs(entry.getKey(), entry.getValue()); + } + + ExpansionApi.ExpansionRequest.Builder requestBuilder = + ExpansionApi.ExpansionRequest.newBuilder(); + ExpansionApi.ExpansionRequest request = + requestBuilder + .setComponents(runnerAPIpipeline.getComponents()) + .setTransform(ptransformBuilder.build()) + .setNamespace(UPGRADE_NAMESPACE) + .build(); + + ExpansionApi.ExpansionResponse response = + clientFactory.getExpansionServiceClient(transformServiceEndpoint).expand(request); + + if (!Strings.isNullOrEmpty(response.getError())) { + throw new RuntimeException(String.format("expansion service error: %s", response.getError())); + } + + Map newEnvironmentsWithDependencies = + response.getComponents().getEnvironmentsMap().entrySet().stream() + .filter( + kv -> + !runnerAPIpipeline.getComponents().getEnvironmentsMap().containsKey(kv.getKey()) + && kv.getValue().getDependenciesCount() != 0) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + RunnerApi.Components expandedComponents = + response + .getComponents() + .toBuilder() + .putAllEnvironments( + External.ExpandableTransform.resolveArtifacts( + newEnvironmentsWithDependencies, transformServiceEndpoint)) + .build(); + RunnerApi.PTransform expandedTransform = response.getTransform(); + List expandedRequirements = response.getRequirementsList(); + + RunnerApi.Components.Builder newComponentsBuilder = expandedComponents.toBuilder(); + + // We record transforms that consume outputs of the old transform and update them to consume + // outputs of the new (upgraded) transform. + Collection oldOutputs = transformToUpgrade.getOutputsMap().values(); + Map inputReplacements = new HashMap<>(); + if (transformToUpgrade.getOutputsMap().size() == 1) { + inputReplacements.put( + oldOutputs.iterator().next(), + expandedTransform.getOutputsMap().values().iterator().next()); + } else { + for (Map.Entry entry : transformToUpgrade.getOutputsMap().entrySet()) { + if (expandedTransform.getOutputsMap().keySet().contains(entry.getKey())) { + throw new IllegalArgumentException( + "Original transform did not have an output with tag " + + entry.getKey() + + " but upgraded transform did."); + } + String newOutput = expandedTransform.getOutputsMap().get(entry.getKey()); + if (newOutput == null) { + throw new IllegalArgumentException( + "Could not find an output with tag " + + entry.getKey() + + " for the transform " + + expandedTransform); + } + inputReplacements.put(entry.getValue(), newOutput); + } + } + + // The list of obsolete (overridden) transforms that should be removed from the pipeline + // produced by this method. + List transformsToRemove = new ArrayList<>(); + recursivelyFindSubTransforms( + transformId, runnerAPIpipeline.getComponents(), transformsToRemove); + + Map updatedExpandedTransformMap = + expandedComponents.getTransformsMap().entrySet().stream() + .filter( + entry -> { + // Do not include already overridden transforms. + return !transformsToRemove.contains(entry.getKey()); + }) + .collect( + Collectors.toMap( + entry -> entry.getKey(), + entry -> { + // Fix inputs + Map inputsMap = entry.getValue().getInputsMap(); + PTransform.Builder transformBuilder = entry.getValue().toBuilder(); + if (!Collections.disjoint(inputsMap.values(), inputReplacements.keySet())) { + Map updatedInputsMap = new HashMap<>(); + for (Map.Entry inputEntry : inputsMap.entrySet()) { + String updaterValue = + inputReplacements.containsKey(inputEntry.getValue()) + ? inputReplacements.get(inputEntry.getValue()) + : inputEntry.getValue(); + updatedInputsMap.put(inputEntry.getKey(), updaterValue); + } + transformBuilder.clearInputs(); + transformBuilder.putAllInputs(updatedInputsMap); + } + return transformBuilder.build(); + })); + + newComponentsBuilder.clearTransforms(); + newComponentsBuilder.putAllTransforms(updatedExpandedTransformMap); + newComponentsBuilder.putTransforms(transformId, expandedTransform); + + RunnerApi.Pipeline.Builder newRunnerAPIPipelineBuilder = runnerAPIpipeline.toBuilder(); + newRunnerAPIPipelineBuilder.clearComponents(); + newRunnerAPIPipelineBuilder.setComponents(newComponentsBuilder.build()); + + newRunnerAPIPipelineBuilder.addAllRequirements(expandedRequirements); + + return newRunnerAPIPipelineBuilder.build(); + } + + private static void recursivelyFindSubTransforms( + String transformId, RunnerApi.Components components, List results) { + results.add(transformId); + PTransform transform = components.getTransformsMap().get(transformId); + if (transform == null) { + throw new IllegalArgumentException("Could not find a transform with id " + transformId); + } + List subTransforms = transform.getSubtransformsList(); + if (subTransforms != null) { + for (String subTransformId : subTransforms) { + recursivelyFindSubTransforms(subTransformId, components, results); + } + } + } + + private static int findAvailablePort() throws IOException { + ServerSocket s = new ServerSocket(0); + try { + return s.getLocalPort(); + } finally { + s.close(); + try { + // Some systems don't free the port for future use immediately. + Thread.sleep(100); + } catch (InterruptedException exn) { + // ignore + } + } + } + + @Override + public void close() throws Exception { + clientFactory.close(); + } +} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 1b3aa50c7b44..294d89308a31 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; import org.checkerframework.checker.nullness.qual.Nullable; @@ -47,7 +46,7 @@ public class WindowIntoTranslation { static class WindowAssignTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(Assign transform) { + public String getUrn() { return PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN; } @@ -116,7 +115,7 @@ public static TransformPayloadTranslator create() { private WindowIntoPayloadTranslator() {} @Override - public String getUrn(Window.Assign transform) { + public String getUrn() { return PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java index cce140536114..3a23ed073776 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -276,7 +276,7 @@ public boolean isRunnerDeterminedSharding() { static class WriteFilesTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(WriteFiles transform) { + public String getUrn() { return WRITE_FILES_TRANSFORM_URN; } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java new file mode 100644 index 000000000000..6620e780bc16 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core.construction; + +import static org.junit.Assert.assertEquals; + +import com.google.auto.service.AutoService; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.charset.Charset; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.beam.model.expansion.v1.ExpansionApi; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.ToString; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for TransformServiceBasedOverride. */ +@RunWith(JUnit4.class) +public class TransformUpgraderTest { + static class TestTransform extends PTransform, PCollection> { + private int testParam; + + public TestTransform(int testParam) { + this.testParam = testParam; + } + + @Override + public PCollection expand(PCollection input) { + return input.apply( + MapElements.via( + new SimpleFunction() { + @Override + public Integer apply(Integer input) { + return input * testParam; + } + })); + } + + public Integer getTestParam() { + return testParam; + } + } + + static class TestTransformPayloadTranslator + implements PTransformTranslation.TransformPayloadTranslator { + + static final String URN = "beam:transform:test:transform_to_update"; + + Schema configRowSchema = Schema.builder().addInt32Field("multiplier").build(); + + @Override + public String getUrn() { + return URN; + } + + @Override + public TestTransform fromConfigRow(Row configRow) { + return new TestTransform(configRow.getInt32("multiplier")); + } + + @Override + public Row toConfigRow(TestTransform transform) { + return Row.withSchema(configRowSchema) + .withFieldValue("multiplier", transform.getTestParam()) + .build(); + } + + @Override + public RunnerApi.@Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) + throws IOException { + + int testParam = application.getTransform().getTestParam(); + + FunctionSpec.Builder specBuilder = FunctionSpec.newBuilder(); + specBuilder.setUrn(getUrn()); + + ByteStringOutputStream byteStringOut = new ByteStringOutputStream(); + ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteStringOut); + objectOutputStream.writeObject(testParam); + objectOutputStream.flush(); + specBuilder.setPayload(byteStringOut.toByteString()); + + return specBuilder.build(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, TestTransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(TestTransform.class, new TestTransformPayloadTranslator()); + } + } + + static class TestTransform2 extends TestTransform { + public TestTransform2(int testParam) { + super(testParam); + } + } + + static class TestTransformPayloadTranslator2 extends TestTransformPayloadTranslator { + static final String URN = "beam:transform:test:transform_to_update2"; + + @Override + public String getUrn() { + return URN; + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar2 implements TransformPayloadTranslatorRegistrar { + @Override + public Map, TestTransformPayloadTranslator2> + getTransformPayloadTranslators() { + return Collections.singletonMap(TestTransform2.class, new TestTransformPayloadTranslator2()); + } + } + + static class TestExpansionServiceClientFactory implements ExpansionServiceClientFactory { + ExpansionApi.ExpansionResponse response; + + @Override + public ExpansionServiceClient getExpansionServiceClient( + Endpoints.ApiServiceDescriptor endpoint) { + return new ExpansionServiceClient() { + @Override + public ExpansionApi.ExpansionResponse expand(ExpansionApi.ExpansionRequest request) { + RunnerApi.Components.Builder responseComponents = request.getComponents().toBuilder(); + RunnerApi.PTransform transformToUpgrade = + request.getComponents().getTransformsMap().get("TransformUpgraderTest-TestTransform"); + ByteString alreadyUpgraded = ByteString.empty(); + try { + alreadyUpgraded = transformToUpgrade.getAnnotationsOrThrow("already_upgraded"); + } catch (Exception e) { + // Ignore + } + if (!alreadyUpgraded.isEmpty()) { + transformToUpgrade = + request + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform2"); + } + if (!transformToUpgrade + .getSpec() + .getUrn() + .equals(request.getTransform().getSpec().getUrn())) { + throw new RuntimeException("Could not find a valid transform to upgrade"); + } + + Integer oldParam; + try { + ByteArrayInputStream byteArrayInputStream = + new ByteArrayInputStream(transformToUpgrade.getSpec().getPayload().toByteArray()); + ObjectInputStream objectInputStream = new ObjectInputStream(byteArrayInputStream); + oldParam = (Integer) objectInputStream.readObject(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + RunnerApi.PTransform.Builder upgradedTransform = transformToUpgrade.toBuilder(); + FunctionSpec.Builder specBuilder = upgradedTransform.getSpecBuilder(); + + ByteStringOutputStream byteStringOutputStream = new ByteStringOutputStream(); + try { + ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteStringOutputStream); + objectOutputStream.writeObject(oldParam * 2); + objectOutputStream.flush(); + specBuilder.setPayload(byteStringOutputStream.toByteString()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + upgradedTransform.setSpec(specBuilder.build()); + upgradedTransform.putAnnotations( + "already_upgraded", + ByteString.copyFrom("dummyvalue".getBytes(Charset.defaultCharset()))); + + response = + ExpansionApi.ExpansionResponse.newBuilder() + .setComponents(responseComponents.build()) + .setTransform(upgradedTransform.build()) + .build(); + return response; + } + + @Override + public ExpansionApi.DiscoverSchemaTransformResponse discover( + ExpansionApi.DiscoverSchemaTransformRequest request) { + return null; + } + + @Override + public void close() throws Exception { + // do nothing + } + }; + } + + @Override + public void close() throws Exception { + // do nothing + } + } + + private void validateTestParam(RunnerApi.PTransform updatedTestTransform, Integer expectedValue) { + Integer updatedParam; + try { + ByteArrayInputStream byteArrayInputStream = + new ByteArrayInputStream(updatedTestTransform.getSpec().getPayload().toByteArray()); + ObjectInputStream objectInputStream = new ObjectInputStream(byteArrayInputStream); + updatedParam = (Integer) objectInputStream.readObject(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + assertEquals(Integer.valueOf(expectedValue), updatedParam); + } + + @Test + public void testTransformUpgrade() throws Exception { + Pipeline pipeline = Pipeline.create(); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(new TestTransform(2)) + .apply(ToString.elements()) + .apply(TextIO.write().to("dummyfilename")); + + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline, false); + ExternalTranslationOptions options = + PipelineOptionsFactory.create().as(ExternalTranslationOptions.class); + List urnsToOverride = ImmutableList.of(TestTransformPayloadTranslator.URN); + options.setTransformsToOverride(urnsToOverride); + options.setTransformServiceAddress("dummyaddress"); + + RunnerApi.Pipeline upgradedPipelineProto = + TransformUpgrader.of(new TestExpansionServiceClientFactory()) + .upgradeTransformsViaTransformService(pipelineProto, urnsToOverride, options); + + RunnerApi.PTransform upgradedTransform = + upgradedPipelineProto + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform"); + + validateTestParam(upgradedTransform, 4); + } + + @Test + public void testTransformUpgradeMultipleOccurrences() throws Exception { + Pipeline pipeline = Pipeline.create(); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(new TestTransform(2)) + .apply(ToString.elements()) + .apply(TextIO.write().to("dummyfilename")); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(new TestTransform(2)) + .apply(ToString.elements()) + .apply(TextIO.write().to("dummyfilename")); + + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline, false); + ExternalTranslationOptions options = + PipelineOptionsFactory.create().as(ExternalTranslationOptions.class); + List urnsToOverride = ImmutableList.of(TestTransformPayloadTranslator.URN); + options.setTransformsToOverride(urnsToOverride); + options.setTransformServiceAddress("dummyaddress"); + + RunnerApi.Pipeline upgradedPipelineProto = + TransformUpgrader.of(new TestExpansionServiceClientFactory()) + .upgradeTransformsViaTransformService(pipelineProto, urnsToOverride, options); + + RunnerApi.PTransform upgradedTransform1 = + upgradedPipelineProto + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform"); + validateTestParam(upgradedTransform1, 4); + + RunnerApi.PTransform upgradedTransform2 = + upgradedPipelineProto + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform2"); + validateTestParam(upgradedTransform2, 4); + } + + @Test + public void testTransformUpgradeMultipleURNs() throws Exception { + Pipeline pipeline = Pipeline.create(); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(new TestTransform(2)) + .apply(ToString.elements()) + .apply(TextIO.write().to("dummyfilename")); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(new TestTransform2(2)) + .apply(ToString.elements()) + .apply(TextIO.write().to("dummyfilename")); + + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline, false); + ExternalTranslationOptions options = + PipelineOptionsFactory.create().as(ExternalTranslationOptions.class); + List urnsToOverride = + ImmutableList.of(TestTransformPayloadTranslator.URN, TestTransformPayloadTranslator2.URN); + options.setTransformsToOverride(urnsToOverride); + options.setTransformServiceAddress("dummyaddress"); + + RunnerApi.Pipeline upgradedPipelineProto = + TransformUpgrader.of(new TestExpansionServiceClientFactory()) + .upgradeTransformsViaTransformService(pipelineProto, urnsToOverride, options); + + RunnerApi.PTransform upgradedTransform1 = + upgradedPipelineProto + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform"); + validateTestParam(upgradedTransform1, 4); + + RunnerApi.PTransform upgradedTransform2 = + upgradedPipelineProto + .getComponents() + .getTransformsMap() + .get("TransformUpgraderTest-TestTransform2"); + validateTestParam(upgradedTransform2, 4); + } +} 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 b725bfbb8d40..6d42d0c3b485 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 @@ -1446,7 +1446,7 @@ private static class CreateStreamingFlinkViewPayloadTranslator private CreateStreamingFlinkViewPayloadTranslator() {} @Override - public String getUrn(CreateStreamingFlinkView.CreateFlinkPCollectionView transform) { + public String getUrn() { return CreateStreamingFlinkView.CREATE_STREAMING_FLINK_VIEW_URN; } } 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 17aea34045ff..26548038a1df 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 @@ -2572,6 +2572,12 @@ public String getUrn(PTransform transform) { return "dataflow_stub:" + transform.getClass().getName(); } + @Override + public String getUrn() { + throw new UnsupportedOperationException( + "URN of DataflowPayloadTranslator depends on the transform. Please use 'getUrn(PTransform transform)' instead."); + } + @Override public RunnerApi.FunctionSpec translate( AppliedPTransform> application, SdkComponents components) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java index facbbb3f1b44..140858d88c04 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java @@ -157,7 +157,7 @@ public static PTransformTranslation.TransformPayloadTranslator create() { private PayloadTranslator() {} @Override - public String getUrn(ParDoSingle transform) { + public String getUrn() { return PAR_DO_TRANSFORM_URN; } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 3c50ae6019f8..078f25e0e38e 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -1632,7 +1632,7 @@ public PCollection expand(PCollection input) { private static class TestTransformTranslator implements TransformPayloadTranslator { @Override - public String getUrn(TestTransform transform) { + public String getUrn() { return "test_transform"; } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java index 9a50d3d579ac..a3ebbffef9a8 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java @@ -59,7 +59,7 @@ static class SamzaPublishViewPayloadTranslator SamzaPublishViewPayloadTranslator() {} @Override - public String getUrn(SamzaPublishView transform) { + public String getUrn() { return SAMZA_PUBLISH_VIEW_URN; } } 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 eaa267375db3..266b67798a22 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 @@ -636,7 +636,7 @@ private static class SparkConsoleIOWriteUnboundedPayloadTranslator ConsoleIO.Write.Unbound> { @Override - public String getUrn(ConsoleIO.Write.Unbound transform) { + public String getUrn() { return ConsoleIO.Write.Unbound.TRANSFORM_URN; } } @@ -645,7 +645,7 @@ private static class SparkCreateStreamPayloadTranslator extends PTransformTranslation.TransformPayloadTranslator.NotSerializable> { @Override - public String getUrn(CreateStream transform) { + public String getUrn() { return CreateStream.TRANSFORM_URN; } } diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 6b52f8d1245e..ec53e3f11e43 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -18,10 +18,12 @@ package org.apache.beam.sdk.expansion.service; import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN; import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import com.google.auto.service.AutoService; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -45,10 +47,12 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.runners.core.construction.Environments; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.core.construction.SplittableParDo; +import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -89,8 +93,10 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Converter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; +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; +import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; @@ -178,6 +184,65 @@ public List getDependencies( } } + List deprecatedTransformURNs = ImmutableList.of(READ_TRANSFORM_URN); + for (TransformPayloadTranslatorRegistrar registrar : + ServiceLoader.load(TransformPayloadTranslatorRegistrar.class)) { + for (Map.Entry, ? extends TransformPayloadTranslator> + entry : registrar.getTransformPayloadTranslators().entrySet()) { + @Initialized TransformPayloadTranslator translator = entry.getValue(); + if (translator == null) { + continue; + } + + String urn; + try { + urn = translator.getUrn(); + if (urn == null) { + LOG.debug( + "Could not load the TransformPayloadTranslator " + + translator + + " to the Expansion Service since it did not produce a unique URN."); + continue; + } + } catch (Exception e) { + LOG.info( + "Could not load the TransformPayloadTranslator " + + translator + + " to the Expansion Service."); + continue; + } + + if (deprecatedTransformURNs.contains(urn)) { + continue; + } + final String finalUrn = urn; + TransformProvider transformProvider = + spec -> { + try { + ExternalConfigurationPayload payload = + ExternalConfigurationPayload.parseFrom(spec.getPayload()); + Row configRow = + RowCoder.of(SchemaTranslation.schemaFromProto(payload.getSchema())) + .decode(new ByteArrayInputStream(payload.getPayload().toByteArray())); + PTransform transformFromRow = translator.fromConfigRow(configRow); + if (transformFromRow != null) { + return transformFromRow; + } else { + throw new RuntimeException( + String.format( + "A transform cannot be initiated using the provided config row %s and the TransformPayloadTranslator %s", + configRow, translator)); + } + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to build transform %s from spec %s", finalUrn, spec), + e); + } + }; + builder.put(finalUrn, transformProvider); + } + } + return builder.build(); } diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index f157cbadee57..25d6b2ac4040 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -29,6 +29,7 @@ dependencies { // :sdks:java:core and transitive dependencies provided project(path: ":model:pipeline", configuration: "shadow") provided project(path: ":sdks:java:core", configuration: "shadow") + provided project(path: ":sdks:java:transform-service:launcher", configuration: "shadow") provided library.java.joda_time provided library.java.slf4j_api provided library.java.vendored_grpc_1_54_0 @@ -80,6 +81,7 @@ dependencies { implementation project(":runners:core-construction-java") implementation project(":runners:core-java") implementation project(":sdks:java:fn-execution") + permitUnusedDeclared project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit testImplementation library.java.mockito_core shadowTestRuntimeClasspath project(path: ":sdks:java:core", configuration: "shadowTest") diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java index c8214529d580..4722a3833fa9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java @@ -52,10 +52,7 @@ static class PubSubReadPayloadTranslator implements TransformPayloadTranslator> { @Override - public String getUrn(Read.Unbounded transform) { - if (!(transform.getSource() instanceof PubsubUnboundedSource.PubsubSource)) { - return null; - } + public String getUrn() { return PTransformTranslation.PUBSUB_READ; } @@ -106,7 +103,7 @@ public RunnerApi.FunctionSpec translate( static class PubSubWritePayloadTranslator implements TransformPayloadTranslator { @Override - public String getUrn(PubsubUnboundedSink.PubsubSink transform) { + public String getUrn() { return PTransformTranslation.PUBSUB_WRITE; } @@ -140,7 +137,7 @@ public RunnerApi.FunctionSpec translate( static class PubSubDynamicWritePayloadTranslator implements TransformPayloadTranslator { @Override - public String getUrn(PubsubUnboundedSink.PubsubDynamicSink transform) { + public String getUrn() { return PTransformTranslation.PUBSUB_WRITE_DYNAMIC; } From 3cdb9fe2b2d8c17d02920ccb3bf81b56695179cf Mon Sep 17 00:00:00 2001 From: Pranav Bhandari Date: Wed, 27 Sep 2023 19:58:09 -0400 Subject: [PATCH 08/64] Bump Python FnAPI beam-master container. (#28704) --- 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 57c0bcdff201..8347a597e968 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-20230915' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20230927' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From 218bda9862407c20adfeffb6723d0de07f2e4db3 Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin Date: Thu, 28 Sep 2023 05:49:24 +0200 Subject: [PATCH 09/64] added jobs to GitHub Actions (#28679) --- .../workflows/beam_PerformanceTests_Cdap.yml | 108 +++++++++++++++++ .../beam_PerformanceTests_HadoopFormat.yml | 108 +++++++++++++++++ .../beam_PerformanceTests_MongoDBIO_IT.yml | 108 +++++++++++++++++ ...manceTests_PubsubIOIT_Python_Streaming.yml | 94 +++++++++++++++ ...beam_PerformanceTests_SparkReceiver_IO.yml | 108 +++++++++++++++++ ...eam_PerformanceTests_TFRecordIOIT_HDFS.yml | 112 ++++++++++++++++++ ...rmanceTests_WordCountIT_PythonVersions.yml | 105 ++++++++++++++++ .../TFRecordIOIT_HDFS.txt | 23 ++++ .../performance-tests-job-configs/cdap.txt | 29 +++++ .../hadoopFormat.txt | 29 +++++ .../mongoDBIO_IT.txt | 28 +++++ .../pubsubIOIT_Python_Streaming.txt | 28 +++++ .../sparkReceiver_IO.txt | 26 ++++ .../wordCountIT_Python.txt | 27 +++++ 14 files changed, 933 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_Cdap.yml create mode 100644 .github/workflows/beam_PerformanceTests_HadoopFormat.yml create mode 100644 .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml create mode 100644 .github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml create mode 100644 .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml create mode 100644 .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml create mode 100644 .github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml create mode 100644 .github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt create mode 100644 .github/workflows/performance-tests-job-configs/cdap.txt create mode 100644 .github/workflows/performance-tests-job-configs/hadoopFormat.txt create mode 100644 .github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt create mode 100644 .github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt create mode 100644 .github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt create mode 100644 .github/workflows/performance-tests-job-configs/wordCountIT_Python.txt diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml new file mode 100644 index 000000000000..0dfd095ecf5f --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -0,0 +1,108 @@ +# 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: PerformanceTests Cdap + +on: + issue_comment: + types: [created] + schedule: + - cron: '13 4/16 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_Cdap_test_arguments_1: '' + +jobs: + beam_PerformanceTests_Cdap: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java CdapIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_Cdap"] + job_phrase: ["Run Java CdapIO Performance Test"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install postgres + id: install_postgres + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml + kubectl wait svc/postgres-for-dev --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=60s + loadbalancer_IP=$(kubectl get svc postgres-for-dev -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo postgres_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/cdap.txt + arguments: | + --postgresServerName=${{ steps.install_postgres.outputs.postgres_IP }} + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:cdap:integrationTest + arguments: | + --info \ + --tests org.apache.beam.sdk.io.cdap.CdapIOIT \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_Cdap_test_arguments_1 }}]' \ diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml new file mode 100644 index 000000000000..c80db8a86a16 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -0,0 +1,108 @@ +# 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: PerformanceTests HadoopFormat + +on: + issue_comment: + types: [created] + schedule: + - cron: '16 7/19 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_HadoopFormat_test_arguments_1: '' + +jobs: + beam_PerformanceTests_HadoopFormat: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java HadoopFormatIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_HadoopFormat"] + job_phrase: ["Run Java HadoopFormatIO Performance Test"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install postgres + id: install_postgres + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml + kubectl wait svc/postgres-for-dev --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=60s + loadbalancer_IP=$(kubectl get svc postgres-for-dev -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo postgres_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/hadoopFormat.txt + arguments: | + --postgresServerName=${{ steps.install_postgres.outputs.postgres_IP }} + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:hadoop-format:integrationTest + arguments: | + --info \ + --tests org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOIT \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_HadoopFormat_test_arguments_1 }}]' \ diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml new file mode 100644 index 000000000000..299d9ac0b77c --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -0,0 +1,108 @@ +# 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: PerformanceTests MongoDBIO IT + +on: + issue_comment: + types: [created] + schedule: + - cron: '14 5/17 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_MongoDBIO_IT_test_arguments_1: '' + +jobs: + beam_PerformanceTests_MongoDBIO_IT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java MongoDBIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_MongoDBIO_IT"] + job_phrase: ["Run Java MongoDBIO Performance Test"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install mongo + id: install_mongo + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/mongodb/load-balancer/mongo.yml + kubectl wait svc/mongo-load-balancer-service --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=60s + loadbalancer_IP=$(kubectl get svc mongo-load-balancer-service -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo mongo_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt + arguments: | + --mongoDBHostName=${{ steps.install_mongo.outputs.mongo_IP }} + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:mongodb:integrationTest + arguments: | + --info \ + --tests org.apache.beam.sdk.io.mongodb.MongoDBIOIT \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_MongoDBIO_IT_test_arguments_1 }}]' \ diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml new file mode 100644 index 000000000000..3a16e482979f --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -0,0 +1,94 @@ +# 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: PerformanceTests PubsubIOIT Python Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '11 2 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_PubsubIOIT_Python_Streaming_test_arguments_1: '' + +jobs: + beam_PerformanceTests_PubsubIOIT_Python_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run PubsubIO Performance Test Python' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_PubsubIOIT_Python_Streaming"] + job_phrase: ["Run PubsubIO Performance Test Python"] + 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: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt + arguments: | + --job_name=performance-tests-psio-python-2gb$(date '+%m%d%H%M%S' --utc) + - name: Run test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.io.gcp.pubsub_io_perf_test \ + -Prunner=TestDataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_PerformanceTests_PubsubIOIT_Python_Streaming_test_arguments_1 }}' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml new file mode 100644 index 000000000000..f2f4dc85ec5e --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -0,0 +1,108 @@ +# 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: PerformanceTests SparkReceiver IO + +on: + issue_comment: + types: [created] + schedule: + - cron: '15 6/18 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_SparkReceiver_IO_test_arguments_1: '' + +jobs: + beam_PerformanceTests_SparkReceiver_IO: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java SparkReceiverIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 120 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_SparkReceiver_IO"] + job_phrase: ["Run Java SparkReceiverIO Performance Test"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install rabbitmq + id: install_rabbitmq + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/rabbit/rabbitmq.yaml + kubectl wait svc/rabbitmq --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=60s + loadbalancer_IP=$(kubectl get svc rabbitmq -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo rabbitmq_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt + arguments: | + --rabbitMqBootstrapServerAddress=amqp://guest:guest@${{ steps.install_rabbitmq.outputs.rabbitmq_IP }}:5672 + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:sparkreceiver:2:integrationTest + arguments: | + --info \ + --tests org.apache.beam.sdk.io.sparkreceiver.SparkReceiverIOIT \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_SparkReceiver_IO_test_arguments_1 }}]' \ diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml new file mode 100644 index 000000000000..bbc4a79aa0f3 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -0,0 +1,112 @@ +# 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: PerformanceTests TFRecordIOIT HDFS + +on: + # TODO(https://github.com/apache/beam/issues/18796) TFRecord performance test is failing only when running on hdfs. + # We need to fix this before enabling this job on jenkins. + # issue_comment: + # types: [created] + # schedule: + # - cron: '17 8/20 * * *' + workflow_dispatch: + +#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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_TFRecordIOIT_HDFS_test_arguments_1: '' + +jobs: + beam_PerformanceTests_TFRecordIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java TFRecordIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_TFRecordIOIT_HDFS"] + job_phrase: ["Run Java TFRecordIO Performance Test HDFS"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=60s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --info \ + --tests org.apache.beam.sdk.io.tfrecord.TFRecordIOIT \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_TFRecordIOIT_HDFS_test_arguments_1 }}]' \ \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml new file mode 100644 index 000000000000..53b157d691c5 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -0,0 +1,105 @@ +# 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: PerformanceTests WordCountIT PythonVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '12 3 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1: '' + +jobs: + beam_PerformanceTests_WordCountIT_PythonVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (startswith(github.event.comment.body, 'Run Python') && + endswith(github.event.comment.body, 'WordCountIT Performance Test')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_PerformanceTests_WordCountIT_PythonVersions"] + job_phrase_1: [Run Python] + job_phrase_2: [WordCountIT Performance Test] + python_version: ['3.8'] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: ${{matrix.python_version}} + - name: Set PY_VER_CLEAN + id: set_py_ver_clean + run: | + PY_VER=${{ matrix.python_version }} + PY_VER_CLEAN=${PY_VER//.} + echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt + arguments: | + --job_name=performance-tests-wordcount-python${{steps.set_py_ver_clean.outputs.py_ver_clean}}-batch-1gb$(date '+%m%d%H%M%S' --utc) + --metrics_table=wordcount_py${{steps.set_py_ver_clean.outputs.py_ver_clean}}_pkb_results + --influx_measurement=wordcount_py${{steps.set_py_ver_clean.outputs.py_ver_clean}}_results + - name: Run test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:runPerformanceTest + arguments: | + --info \ + -Ptest=apache_beam/examples/wordcount_it_test.py::WordCountIT::test_wordcount_it \ + "-Ptest-pipeline-options=${{ env.beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1 }}" \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt new file mode 100644 index 000000000000..608e11c7847a --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt @@ -0,0 +1,23 @@ +# 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. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--numberOfRecords=18000000 +--expectedHash=543104423f8b6eb097acb9f111c19fe4 +--datasetSize=1019380000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/cdap.txt b/.github/workflows/performance-tests-job-configs/cdap.txt new file mode 100644 index 000000000000..60c98a1065d5 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/cdap.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--numberOfRecords=5000000 +--bigQueryDataset=beam_performance +--bigQueryTable=cdapioit_results +--influxMeasurement=cdapioit_results +--postgresUsername=postgres +--postgresPassword=uuinkks +--postgresDatabaseName=postgres +--postgresSsl=false +--postgresPort=5432 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/hadoopFormat.txt b/.github/workflows/performance-tests-job-configs/hadoopFormat.txt new file mode 100644 index 000000000000..712f29e3d52c --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/hadoopFormat.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--numberOfRecords=5000000 +--bigQueryDataset=beam_performance +--bigQueryTable=hadoopformatioit_results +--influxMeasurement=hadoopformatioit_results +--postgresUsername=postgres +--postgresPassword=uuinkks +--postgresDatabaseName=postgres +--postgresSsl=false +--postgresPort=5432 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt b/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt new file mode 100644 index 000000000000..67a60f6407f9 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--numberOfRecords=10000000 +--bigQueryDataset=beam_performance +--bigQueryTable=mongodbioit_results +--influxMeasurement=mongodbioit_results +--mongoDBDatabaseName=beam +--mongoDBPort=27017 +--mongoDBUsername=root +--mongoDBPassword=uuinkkS +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt b/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt new file mode 100644 index 000000000000..257fc196de7e --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=beam_performance +--metrics_table=psio_io_2GB_results +--influx_measurement=python_psio_2GB_results +--input_options=''{\\"num_records\\":2097152,\\"key_size\\":1,\\"value_size\\":1024,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=5 +--autoscaling_algorithm=NONE +--pubsub_namespace_prefix=pubsub_io_performance_ +--wait_until_finish_duration=720000 +--runner=TestDataflowRunner \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt b/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt new file mode 100644 index 000000000000..12d2bde67327 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt @@ -0,0 +1,26 @@ +# 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. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--sourceOptions=''{\"numRecords\":\"5000000\",\"keySizeBytes\":\"1\",\"valueSizeBytes\":\"90\"}'' +--bigQueryDataset=beam_performance +--bigQueryTable=sparkreceiverioit_results +--influxMeasurement=sparkreceiverioit_results +--streamName=rabbitMqTestStream +--readTimeout=1800 +--numWorkers=1 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt b/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt new file mode 100644 index 000000000000..9b9abeeb092a --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--staging_location=gs://temp-storage-for-end-to-end-tests/staging-it +--temp_location=gs://temp-storage-for-end-to-end-tests/temp-it +--runner=TestDataflowRunner +--publish_to_big_query=true +--metrics_dataset=beam_performance +--input=gs://apache-beam-samples/input_small_files/ascii_sort_1MB_input.0000* +--output=gs://temp-storage-for-end-to-end-tests/py-it-cloud/output +--expect_checksum=ea0ca2e5ee4ea5f218790f28d0b9fe7d09d8d710 +--num_workers=10 +--autoscaling_algorithm=NONE \ No newline at end of file From b74a0dc8e6592503909530850f1fb6ec826f4a65 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Thu, 28 Sep 2023 07:50:22 +0400 Subject: [PATCH 10/64] Add Load Tests Combine Dataflow Batch Java workflow (#28676) --- ..._LoadTests_Java_Combine_Dataflow_Batch.yml | 104 ++++++++++++++++++ ...config_Combine_Java_Dataflow_Batch_10b.txt | 30 +++++ ...g_Combine_Java_Dataflow_Batch_Fanout_4.txt | 30 +++++ ...g_Combine_Java_Dataflow_Batch_Fanout_8.txt | 30 +++++ 4 files changed, 194 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt create mode 100644 .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml new file mode 100644 index 000000000000..758d196f273c --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -0,0 +1,104 @@ +# 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: Load Tests Combine Dataflow Batch Java + +on: + issue_comment: + types: [created] + schedule: + - cron: '35 7 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_Combine_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java Combine Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_Combine_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java Combine Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt + - name: run Combine Dataflow Batch Java Load Test 1 (10 b records) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Batch_test_arguments_1 }}' \ + - name: run Combine Dataflow Batch Java Load Test 2 (fanout 4) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Batch_test_arguments_2 }}' \ + - name: run Combine Dataflow Batch Java Load Test 3 (fanout 8) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Batch_test_arguments_3 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt new file mode 100644 index 000000000000..be6edd016dee --- /dev/null +++ b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_Combine_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--topCount=20 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt new file mode 100644 index 000000000000..4f6216c1d8d2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_Combine_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--topCount=20 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt new file mode 100644 index 000000000000..68bed0bae78a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_Combine_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--topCount=20 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=DataflowRunner \ No newline at end of file From e53b68b368d8d19441427134056d688bdd31c135 Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Thu, 28 Sep 2023 17:24:20 +0400 Subject: [PATCH 11/64] Add GitHub Workflow Replacements for Jenkins job_PerformanceTests_SpannerIO and job_PerformanceTests_SQLBigQueryIO_Batch_Java (#28555) --- ...formanceTests_SQLBigQueryIO_Batch_Java.yml | 104 ++++++++++++++++++ ...ormanceTests_SpannerIO_Read_2GB_Python.yml | 94 ++++++++++++++++ ...Tests_SpannerIO_Write_2GB_Python_Batch.yml | 94 ++++++++++++++++ .../SQLBigQueryIO_Batch_Java.txt | 24 ++++ .../spannerIO_Read_2GB_Python.txt | 28 +++++ .../spannerIO_Write_2GB_Python.txt | 28 +++++ 6 files changed, 372 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml create mode 100644 .github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml create mode 100644 .github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml create mode 100644 .github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt create mode 100644 .github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt create mode 100644 .github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml new file mode 100644 index 000000000000..06014a56e682 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -0,0 +1,104 @@ +# 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: PerformanceTests SQLBigQueryIO Batch Java + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 7,19 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1: '' + +jobs: + beam_PerformanceTests_SQLBigQueryIO_Batch_Java: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run SQLBigQueryIO Batch Performance Test Java' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_SQLBigQueryIO_Batch_Java"] + job_phrase: ["Run SQLBigQueryIO Batch Performance Test Java"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt + - name: Run SQLBigQueryIO Batch Performance Test Java + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:extensions:sql:perf-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryIOPushDownIT \ + -DintegrationTestRunner=dataflow \ + '-DintegrationTestPipelineOptions=[${{env.beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1}}]' \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit 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: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml new file mode 100644 index 000000000000..9e464ef58900 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -0,0 +1,94 @@ +# 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: PerformanceTests SpannerIO Read 2GB Python + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 4 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_SpannerIO_Read_2GB_Python_test_arguments_1: '' + +jobs: + beam_PerformanceTests_SpannerIO_Read_2GB_Python: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run SpannerIO Read 2GB Performance Test Python' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 480 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_SpannerIO_Read_2GB_Python"] + job_phrase: ["Run SpannerIO Read 2GB Performance Test Python"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt + arguments: | + --job_name=performance-tests-spanner-read-python-2gb$(date '+%m%d%H%M%S' --utc) + - name: run Performance SpannerIO Read 2GB Test Python + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.io.gcp.experimental.spannerio_read_perf_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.args='${{env.beam_PerformanceTests_SpannerIO_Read_2GB_Python_test_arguments_1}}' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml new file mode 100644 index 000000000000..8cfce643f749 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -0,0 +1,94 @@ +# 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: PerformanceTests SpannerIO Write 2GB Python Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 5 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch_test_arguments_1: '' + +jobs: + beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run SpannerIO Write 2GB Performance Test Python Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 480 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch"] + job_phrase: ["Run SpannerIO Write 2GB Performance Test Python Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt + arguments: | + --job_name=performance-tests-spannerio-write-python-batch-2gb$(date '+%m%d%H%M%S' --utc) + - name: run Performance SpannerIO Write 2GB Test Python + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.io.gcp.experimental.spannerio_write_perf_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.args='${{env.beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch_test_arguments_1}}' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt b/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt new file mode 100644 index 000000000000..2411e6288680 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt @@ -0,0 +1,24 @@ +# 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. + +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--tempRoot=gs://temp-storage-for-perf-tests/loadtests +--metricsBigQueryDataset=beam_performance +--metricsBigQueryTable=sql_bqio_read_java_batch +--runner=DataflowRunner +--maxNumWorkers=5 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt b/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt new file mode 100644 index 000000000000..8ce05ac6a4f3 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-west1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--spanner_instance=beam-test +--spanner_database=pyspanner_read_2gb +--publish_to_big_query=true +--metrics_dataset=beam_performance +--metrics_table=pyspanner_read_2GB_results +--influx_measurement=python_spannerio_read_2GB_results +--input_options=''{\\"num_records\\":2097152,\\"key_size\\":1,\\"value_size\\":1024,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt b/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt new file mode 100644 index 000000000000..de723a9509e1 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-west1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--spanner_instance=beam-test +--spanner_database=pyspanner_write_2gb +--publish_to_big_query=true +--metrics_dataset=beam_performance +--metrics_table=pyspanner_write_2GB_results +--influx_measurement=python_spanner_write_2GB_results +--input_options=''{\\"num_records\\":2097152,\\"key_size\\":1,\\"value_size\\":1024,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file From 941e77d295a41ef55e01448bf46dcd4e35a97c4c Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Thu, 28 Sep 2023 19:46:32 +0600 Subject: [PATCH 12/64] beam_PostCommit_Java_Tpcds (#28495) --- .github/workflows/README.md | 5 +- .../beam_PostCommit_Java_Tpcds_Dataflow.yml | 103 ++++++++++++++++++ .../beam_PostCommit_Java_Tpcds_Flink.yml | 100 +++++++++++++++++ .../beam_PostCommit_Java_Tpcds_Spark.yml | 99 +++++++++++++++++ .../beam_PostCommit_Website_Test.yml | 2 +- 5 files changed, 307 insertions(+), 2 deletions(-) create mode 100644 .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml create mode 100644 .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml create mode 100644 .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 7a77c7774713..f360bc15ae28 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -223,7 +223,10 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![PostCommit Java PVR Spark3 Streaming](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | | [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![PostCommit Java PVR Spark Batch](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | | [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`Run Java Sickbay`| [![.github/workflows/beam_PostCommit_Java_Sickbay.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | -| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | +| [ PostCommit Java Tpcds Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | N/A |`Run Dataflow Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | +| [ PostCommit Java Tpcds Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | N/A |`Run Flink Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | +| [ PostCommit Java Tpcds Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | N/A |`Run Spark Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | +| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml) | | [ PostCommit Java ValidatesRunner Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | N/A |`Run Dataflow Streaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | | [ PostCommit Java ValidatesRunner Dataflow V2 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | | [ PostCommit Java ValidatesRunner Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml new file mode 100644 index 000000000000..6a458e174376 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -0,0 +1,103 @@ +# 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 Java Tpcds Dataflow + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GRADLE_COMMAND_ARGUMENTS: | + --runner=DataflowRunner + --region=us-central1 + --numWorkers=4 + --maxNumWorkers=4 + --autoscalingAlgorithm=NONE + --dataSize=1GB + --sourceType=PARQUET + --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned + --resultsDirectory=gs://beam-tpcds/results/dataflow/ + --tpcParallel=1 + tpcdsBigQueryArgs: | + --bigQueryTable=tpcds + --bigQueryDataset=tpcds + --project=apache-beam-testing + --resourceNameMode=QUERY_RUNNER_AND_MODE + --exportSummaryToBigQuery=true + --tempLocation=gs://temp-storage-for-perf-tests/tpcds + tpcdsInfluxDBArgs: | + --influxDatabase=beam_test_metrics + --influxHost=http://10.128.0.96:8086 + --baseInfluxMeasurement=tpcds + --exportSummaryToInfluxDB=true + --influxRetentionPolicy=forever + tpcdsQueriesArg: 3,7,10,25,26,29,35,38,40,42,43,52,55,69,79,83,84,87,93,96 + +jobs: + beam_PostCommit_Java_Tpcds_Dataflow: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Dataflow Runner Tpcds Tests' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Java_Tpcds_Dataflow"] + job_phrase: ["Run Dataflow Runner Tpcds Tests"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run PostCommit Java Tpcds Dataflow script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:tpcds:run + arguments: | + -Ptpcds.runner=:runners:google-cloud-dataflow-java \ + "-Ptpcds.args=${{env.tpcdsBigQueryArgs}} ${{env.tpcdsInfluxDBArgs}} ${{ env.GRADLE_COMMAND_ARGUMENTS }} --queries=${{env.tpcdsQueriesArg}}" \ diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml new file mode 100644 index 000000000000..e062b108aef6 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -0,0 +1,100 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PostCommit Java Tpcds Flink + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GRADLE_COMMAND_ARGUMENTS: | + --runner=FlinkRunner + --parallelism=4 + --dataSize=1GB + --sourceType=PARQUET + --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned + --resultsDirectory=gs://beam-tpcds/results/flink/ + --tpcParallel=1 + tpcdsBigQueryArgs: | + --bigQueryTable=tpcds + --bigQueryDataset=tpcds + --project=apache-beam-testing + --resourceNameMode=QUERY_RUNNER_AND_MODE + --exportSummaryToBigQuery=true + --tempLocation=gs://temp-storage-for-perf-tests/tpcds + tpcdsInfluxDBArgs: | + --influxDatabase=beam_test_metrics + --influxHost=http://10.128.0.96:8086 + --baseInfluxMeasurement=tpcds + --exportSummaryToInfluxDB=true + --influxRetentionPolicy=forever + tpcdsQueriesArg: 3,7,10,25,26,29,35,38,40,42,43,52,55,69,79,83,84,87,93,96 + +jobs: + beam_PostCommit_Java_Tpcds_Flink: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Flink Runner Tpcds Tests' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Java_Tpcds_Flink"] + job_phrase: ["Run Flink Runner Tpcds Tests"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run PostCommit Java Tpcds Flink script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:tpcds:run + arguments: | + -Ptpcds.runner=:runners:flink:1.13 \ + "-Ptpcds.args=${{env.tpcdsBigQueryArgs}} ${{env.tpcdsInfluxDBArgs}} ${{ env.GRADLE_COMMAND_ARGUMENTS }} --queries=${{env.tpcdsQueriesArg}}" \ diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml new file mode 100644 index 000000000000..6dc6cbd5f1a0 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -0,0 +1,99 @@ +# 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 Java Tpcds Spark + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GRADLE_COMMAND_ARGUMENTS: | + --runner=SparkRunner + --dataSize=1GB + --sourceType=PARQUET + --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned + --resultsDirectory=gs://beam-tpcds/results/spark3-rdd/ + --tpcParallel=1 + tpcdsBigQueryArgs: | + --bigQueryTable=tpcds + --bigQueryDataset=tpcds + --project=apache-beam-testing + --resourceNameMode=QUERY_RUNNER_AND_MODE + --exportSummaryToBigQuery=true + --tempLocation=gs://temp-storage-for-perf-tests/tpcds + tpcdsInfluxDBArgs: | + --influxDatabase=beam_test_metrics + --influxHost=http://10.128.0.96:8086 + --baseInfluxMeasurement=tpcds + --exportSummaryToInfluxDB=true + --influxRetentionPolicy=forever + tpcdsQueriesArg: 3,7,10,25,26,29,35,38,40,42,43,52,55,69,79,83,84,87,93,96 + +jobs: + beam_PostCommit_Java_Tpcds_Spark: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Spark Runner Tpcds Tests' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Java_Tpcds_Spark"] + job_phrase: ["Run Spark Runner Tpcds Tests"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run PostCommit Java Tpcds Spark script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:tpcds:run + arguments: | + -Ptpcds.runner=:runners:spark:3 \ + "-Ptpcds.args=${{env.tpcdsBigQueryArgs}} ${{env.tpcdsInfluxDBArgs}} ${{ env.GRADLE_COMMAND_ARGUMENTS }} --queries=${{env.tpcdsQueriesArg}}" \ diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index 6155a45ef6a6..b186e8ffcd81 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -55,7 +55,7 @@ jobs: github.event_name == 'schedule' || github.event.comment.body == 'Run Full Website Test' runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 30 + timeout-minutes: 60 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: From f824adcae1fd4495c1478631ea0c2a40568e8061 Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Thu, 28 Sep 2023 17:58:17 +0400 Subject: [PATCH 13/64] Add GitHub Workflow Replacements for Jenkins job_PerformanceTests_JDBC (#28602) * Add GitHub Workflow Replacements for Jenkins job_PerformanceTests_JDBC * Add fix for beam_PerformanceTests_BiqQueryIO_Write_Python_Batch and beam_PerformanceTests_BiqQueryIO_Read_Python --- ...erformanceTests_BiqQueryIO_Read_Python.yml | 35 +++--- ...nceTests_BiqQueryIO_Write_Python_Batch.yml | 33 +++--- .../workflows/beam_PerformanceTests_JDBC.yml | 109 ++++++++++++++++++ .../performance-tests-job-configs/JDBC.txt | 29 +++++ ..._Python.txt => biqQueryIO_Read_Python.txt} | 11 +- ....txt => biqQueryIO_Write_Python_Batch.txt} | 11 +- 6 files changed, 175 insertions(+), 53 deletions(-) create mode 100644 .github/workflows/beam_PerformanceTests_JDBC.yml create mode 100644 .github/workflows/performance-tests-job-configs/JDBC.txt rename .github/workflows/performance-tests-job-configs/{config_PerformanceTests_BiqQueryIO_Read_Python.txt => biqQueryIO_Read_Python.txt} (76%) rename .github/workflows/performance-tests-job-configs/{config_PerformanceTests_BiqQueryIO_Write_Python.txt => biqQueryIO_Write_Python_Batch.txt} (76%) diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index 224689ee9086..58958de69c89 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance BigQueryIO Read Python +name: PerformanceTests BiqQueryIO Read Python on: issue_comment: @@ -47,6 +47,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_BiqQueryIO_Read_Python_test_arguments_1: '' jobs: beam_PerformanceTests_BiqQueryIO_Read_Python: @@ -69,24 +70,20 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' - # This code block prepares configuration data for a task. - - name: Prepare config - id: set_config - shell: bash - run: | - # Capture the current date and time in a specific format - CURDATE=$(date '+%m%d%H%M%S' --utc) - # Reads the configurations file and exclude lines starting with # and then remove line breaks - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Read_Python.txt | tr '\n' ' ') - # Appends the value of CURDATE to the original configuration - CONFIGWITHDATE=$(echo "${CURCONFIG/bqio_read_python_/bqio_read_python_$CURDATE}") - # Assigns $CONFIGWITHDATE to a variable named prepared_config - echo "prepared_config=$CONFIGWITHDATE" >> $GITHUB_OUTPUT - - name: run BigQueryIO Read Python Performance Test + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt + arguments: | + --job_name=performance-tests-bqio-read-python-10gb$(date '+%m%d%H%M%S' --utc) + - name: Run BigQueryIO Read Performance Test Python uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:apache_beam:testing:load_tests:run @@ -94,4 +91,4 @@ jobs: -PloadTest.mainClass=apache_beam.io.gcp.bigquery_read_perf_test \ -PpythonVersion=3.8 \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{steps.set_config.outputs.prepared_config}}' \ No newline at end of file + '-PloadTest.args=${{env.beam_PerformanceTests_BiqQueryIO_Read_Python_test_arguments_1}}' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index 24cd1b25ca73..f0fcd20bd3b5 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance BigQueryIO Write Python Batch +name: PerformanceTests BiqQueryIO Write Python Batch on: issue_comment: @@ -47,6 +47,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_BiqQueryIO_Write_Python_Batch_test_arguments_1: '' jobs: beam_PerformanceTests_BiqQueryIO_Write_Python_Batch: @@ -69,23 +70,19 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' - # This code block prepares configuration data for a task. - - name: Prepare config - id: set_config - shell: bash - run: | - # capture the current date and time in a specific format - CURDATE=$(date '+%m%d%H%M%S' --utc) - # reads the configurations file and exclude lines starting with # and then remove line breaks - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Write_Python.txt | tr '\n' ' ') - # appends the value of CURDATE to the original configuration - CONFIGWITHDATE=$(echo "${CURCONFIG/bqio_write_python_/bqio_write_python_$CURDATE}") - # assigns $CONFIGWITHDATE to a variable named prepared_config - echo "prepared_config=$CONFIGWITHDATE" >> $GITHUB_OUTPUT + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt + arguments: | + --job_name=performance-tests-bqio-write-python-batch-10gb$(date '+%m%d%H%M%S' --utc) - name: run BigQueryIO Write Batch Python Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -94,4 +91,4 @@ jobs: -PloadTest.mainClass=apache_beam.io.gcp.bigquery_write_perf_test \ -PpythonVersion=3.8 \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{steps.set_config.outputs.prepared_config}}' \ No newline at end of file + '-PloadTest.args=${{env.beam_PerformanceTests_BiqQueryIO_Write_Python_Batch_test_arguments_1}}' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml new file mode 100644 index 000000000000..c5995480e9d5 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -0,0 +1,109 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PerformanceTests JDBC + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 1,13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_JDBC_test_arguments_1: '' + +jobs: + beam_PerformanceTests_JDBC: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java JdbcIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_JDBC"] + job_phrase: ["Run Java JdbcIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Postgres for dev + id: postgres_for_dev + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml + kubectl wait svc/postgres-for-dev --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_ip=$(kubectl get svc postgres-for-dev -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo "pfd_ip=$loadbalancer_ip" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/JDBC.txt + arguments: | + --postgresServerName=${{ steps.postgres_for_dev.outputs.pfd_ip }} + - name: Run Java JdbcIO Performance Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:jdbc:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.jdbc.JdbcIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{env.beam_PerformanceTests_JDBC_test_arguments_1}}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/JDBC.txt b/.github/workflows/performance-tests-job-configs/JDBC.txt new file mode 100644 index 000000000000..74fd85145c59 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/JDBC.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--numberOfRecords=5000000 +--bigQueryDataset=beam_performance +--bigQueryTable=jdbcioit_results +--influxMeasurement=jdbcioit_results +--postgresUsername=postgres +--postgresPassword=uuinkks +--postgresDatabaseName=postgres +--postgresSsl=false +--postgresPort=5432 +--autoscalingAlgorithm=NONE +--numWorkers=5 \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Read_Python.txt b/.github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt similarity index 76% rename from .github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Read_Python.txt rename to .github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt index facf8b91ddb4..12428da8d091 100644 --- a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Read_Python.txt +++ b/.github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # 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. -############################################################################### ---job_name=performance-tests-bqio-read-python-10gb0917155348 ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --input_dataset=beam_performance @@ -25,9 +22,7 @@ --metrics_dataset=beam_performance --metrics_table=bqio_read_10GB_results --influx_measurement=python_bqio_read_10GB_results ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --input_options=''{\\"num_records\\":10485760,\\"key_size\\":1,\\"value_size\\":1024,\\"algorithm\\":\\"lcg\\"}'' ---num_****s=5 +--num_workers=5 --autoscaling_algorithm=NONE --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Write_Python.txt b/.github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt similarity index 76% rename from .github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Write_Python.txt rename to .github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt index b8c6bd9265f0..9edec48d3d23 100644 --- a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_BiqQueryIO_Write_Python.txt +++ b/.github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # 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. -############################################################################### ---job_name=performance-tests-bqio-write-python-batch-10gb0917155348 ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --output_dataset=beam_performance @@ -25,9 +22,7 @@ --metrics_dataset=beam_performance --metrics_table=bqio_write_10GB_results --influx_measurement=python_bqio_write_10GB_results ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --input_options=''{\\"num_records\\":10485760,\\"key_size\\":1,\\"value_size\\":1024,\\"algorithm\\":\\"lcg\\"}'' ---num_****s=5 +--num_workers=5 --autoscaling_algorithm=NONE --runner=DataflowRunner \ No newline at end of file From 072848f069d89bcdfcd873baaae07cf5ab84129f Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Thu, 28 Sep 2023 17:59:56 +0400 Subject: [PATCH 14/64] Add Load Tests Combine/ParDo SparkStructuredStreaming Batch Java workflows (#28714) * Add Load Tests Combine/ParDo SparkStructuredStreaming Batch Java workflows * Refactoring --- ...Combine_SparkStructuredStreaming_Batch.yml | 107 ++++++++++++++++ ...a_ParDo_SparkStructuredStreaming_Batch.yml | 117 ++++++++++++++++++ ...ine_SparkStructuredStreaming_Batch_10b.txt | 27 ++++ ...parkStructuredStreaming_Batch_Fanout_4.txt | 27 ++++ ...parkStructuredStreaming_Batch_Fanout_8.txt | 27 ++++ ...StructuredStreaming_Batch_100_counters.txt | 26 ++++ ...kStructuredStreaming_Batch_10_counters.txt | 26 ++++ ...parkStructuredStreaming_Batch_10_times.txt | 26 ++++ ...arkStructuredStreaming_Batch_200_times.txt | 26 ++++ 9 files changed, 409 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml new file mode 100644 index 000000000000..28c17ffb535c --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -0,0 +1,107 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java Combine SparkStructuredStreaming Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '15 18 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_1: '' + beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_2: '' + beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_3: '' + +jobs: + beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java Combine SparkStructuredStreaming Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch"] + job_phrase: ["Run Load Tests Java Combine SparkStructuredStreaming Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt + - name: run Combine SparkStructuredStreaming Batch Java Load Test 1 (10b records) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_1 }}' \ + - name: run Combine SparkStructuredStreaming Batch Java Load Test 2 (fanout 4) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_2 }}' \ + - name: run Combine SparkStructuredStreaming Batch Java Load Test 3 (fanout 8) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_3 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml new file mode 100644 index 000000000000..afbb79caec91 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -0,0 +1,117 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java ParDo SparkStructuredStreaming Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '25 8 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_1: '' + beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_2: '' + beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_3: '' + beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_4: '' + +jobs: + beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java ParDo SparkStructuredStreaming Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch"] + job_phrase: ["Run Load Tests Java ParDo SparkStructuredStreaming Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt + - name: run ParDo SparkStructuredStreaming Batch Java Load Test 1 (10 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_1 }}' \ + - name: run ParDo SparkStructuredStreaming Batch Java Load Test 2 (200 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_2 }}' \ + - name: run ParDo SparkStructuredStreaming Batch Java Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_3 }}' \ + - name: run ParDo SparkStructuredStreaming Batch Java Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt new file mode 100644 index 000000000000..a8e93ca4c702 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_Combine_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--topCount=20 +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt new file mode 100644 index 000000000000..c0240410a356 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_Combine_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--topCount=20 +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt new file mode 100644 index 000000000000..17daad210ed9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_Combine_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_combine_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--topCount=20 +--perKeyCombiner=TOP_LARGEST +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt new file mode 100644 index 000000000000..5c2ccf18f3cb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_ParDo_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=100 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt new file mode 100644 index 000000000000..9d0b1c65a2ed --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_ParDo_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=10 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt new file mode 100644 index 000000000000..d3a82cf3caae --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_ParDo_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=10 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt new file mode 100644 index 000000000000..e5642d9b3246 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_ParDo_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=200 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file From c3cfccae1e10ba6a073ee23dce58e4671c543cf8 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Thu, 28 Sep 2023 19:08:10 +0400 Subject: [PATCH 15/64] Add Load Tests ParDo Dataflow Java workflows (#28713) * Add Load Tests ParDo Dataflow Java workflows * Refactoring --- ...am_LoadTests_Java_ParDo_Dataflow_Batch.yml | 117 ++++++++++++++++++ ...oadTests_Java_ParDo_Dataflow_Streaming.yml | 117 ++++++++++++++++++ ...java_ParDo_Dataflow_Batch_100_counters.txt | 29 +++++ .../java_ParDo_Dataflow_Batch_10_counters.txt | 29 +++++ .../java_ParDo_Dataflow_Batch_10_times.txt | 29 +++++ .../java_ParDo_Dataflow_Batch_200_times.txt | 30 +++++ ..._ParDo_Dataflow_Streaming_100_counters.txt | 30 +++++ ...a_ParDo_Dataflow_Streaming_10_counters.txt | 30 +++++ ...java_ParDo_Dataflow_Streaming_10_times.txt | 30 +++++ ...ava_ParDo_Dataflow_Streaming_200_times.txt | 30 +++++ 10 files changed, 471 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml new file mode 100644 index 000000000000..6bd52d2ebae0 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -0,0 +1,117 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java ParDo Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '55 9 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_1: '' + beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_2: '' + beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_3: '' + beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_4: '' + +jobs: + beam_LoadTests_Java_ParDo_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java ParDo Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_ParDo_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java ParDo Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt + - name: run ParDo Dataflow Batch Java Load Test 1 (10 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_1 }}' \ + - name: run ParDo Dataflow Batch Java Load Test 2 (200 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_2 }}' \ + - name: run ParDo Dataflow Batch Java Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_3 }}' \ + - name: run ParDo Dataflow Batch Java Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml new file mode 100644 index 000000000000..dbee457b6ec5 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -0,0 +1,117 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java ParDo Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 11 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_1: '' + beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_2: '' + beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_3: '' + beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_4: '' + +jobs: + beam_LoadTests_Java_ParDo_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java ParDo Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_ParDo_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Java ParDo Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt + - name: run ParDo Dataflow Streaming Java Load Test 1 (10 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_1 }}' \ + - name: run ParDo Dataflow Streaming Java Load Test 2 (200 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_2 }}' \ + - name: run ParDo Dataflow Streaming Java Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_3 }}' \ + - name: run ParDo Dataflow Streaming Java Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt new file mode 100644 index 000000000000..b18650a8e8a6 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_ParDo_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=100 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt new file mode 100644 index 000000000000..c6a7785e02c4 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_ParDo_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=10 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt new file mode 100644 index 000000000000..8e692050c25c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_ParDo_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=10 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt new file mode 100644 index 000000000000..795990ee7ba0 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_ParDo_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=200 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt new file mode 100644 index 000000000000..adcfb461913d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_ParDo_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=100 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt new file mode 100644 index 000000000000..a954c0b8180e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_ParDo_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=10 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt new file mode 100644 index 000000000000..9b579c428be1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_ParDo_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=10 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt new file mode 100644 index 000000000000..6b08284cd06e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_ParDo_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=200 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file From 052d2644c1751630ea733aaa63ad9d10b51fa898 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Thu, 28 Sep 2023 19:22:02 +0400 Subject: [PATCH 16/64] Add Load Tests Combine Dataflow Streaming Java workflow (#28677) * Add Load Tests Combine Dataflow Streaming Java workflow * Update cron * Refactoring --- ...dTests_Java_Combine_Dataflow_Streaming.yml | 107 ++++++++++++++++++ .../java_Combine_Dataflow_Streaming_10b.txt | 31 +++++ ...va_Combine_Dataflow_Streaming_Fanout_4.txt | 31 +++++ ...va_Combine_Dataflow_Streaming_Fanout_8.txt | 31 +++++ 4 files changed, 200 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml new file mode 100644 index 000000000000..b0df52354722 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -0,0 +1,107 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java Combine Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '25 14 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_1: '' + beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_2: '' + beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_3: '' + +jobs: + beam_LoadTests_Java_Combine_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java Combine Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_Combine_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Java Combine Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt + - name: run Combine Dataflow Streaming Java Load Test 1 (10 b records) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_1 }}' \ + - name: run Combine Dataflow Streaming Java Load Test 2 (fanout 4) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_2 }}' \ + - name: run Combine Dataflow Streaming Java Load Test 3 (fanout 8) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_3 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt new file mode 100644 index 000000000000..f52b737aaa94 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_Combine_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_combine_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--topCount=20 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt new file mode 100644 index 000000000000..0a6424d3becc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_Combine_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_combine_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--topCount=20 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt new file mode 100644 index 000000000000..9425a442cda2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_Combine_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_combine_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--topCount=20 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--perKeyCombiner=TOP_LARGEST +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file From 50f2b116596b7dcadd02afc8b52a962a0eba35ae Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Thu, 28 Sep 2023 19:25:00 +0400 Subject: [PATCH 17/64] Add GitHub Workflow Replacements for Jenkins job_PerformanceTests_Kafka_IO (#28712) --- .../beam_PerformanceTests_Kafka_IO.yml | 122 ++++++++++++++++++ .../kafka_IO_Batch.txt | 26 ++++ .../kafka_IO_Streaming.txt | 27 ++++ 3 files changed, 175 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_Kafka_IO.yml create mode 100644 .github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt create mode 100644 .github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml new file mode 100644 index 000000000000..ea709238b8f3 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -0,0 +1,122 @@ +# 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: PerformanceTests Kafka IO + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 2,14 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_Kafka_IO_test_arguments_1: '' + beam_PerformanceTests_Kafka_IO_test_arguments_2: '' + +jobs: + beam_PerformanceTests_Kafka_IO: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java KafkaIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 120 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_Kafka_IO"] + job_phrase: ["Run Java KafkaIO Performance Test"] + env: + KAFKA_SERVICE_PORT: 32400 + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Kafka + id: install_kafka + run: | + cd ${{ github.workspace }}/.test-infra/kubernetes/kafka-cluster/ + kubectl apply -R -f ${{ github.workspace }}/.test-infra/kubernetes/kafka-cluster/ + - name: Set up Kafka brokers + id: set_brokers + run: | + declare -a kafka_service_brokers + for INDEX in {0..2}; do + kubectl wait svc/outside-${INDEX} --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + kafka_service_brokers[$INDEX]=$(kubectl get svc outside-${INDEX} -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo "KAFKA_SERVICE_BROKER_${INDEX}=${kafka_service_brokers[$INDEX]}" >> $GITHUB_OUTPUT + done + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt + arguments: | + --kafkaBootstrapServerAddresses=${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_0 }}:${{ env.KAFKA_SERVICE_PORT }},${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_1 }}:${{ env.KAFKA_SERVICE_PORT }},${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_2 }}:${{ env.KAFKA_SERVICE_PORT }} + - name: Run Java KafkaIO Performance Streaming Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:kafka:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.kafka.KafkaIOIT.testKafkaIOReadsAndWritesCorrectlyInStreaming \ + --info \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{env.beam_PerformanceTests_Kafka_IO_test_arguments_1}}]' \ + - name: Run Java KafkaIO Performance Batch Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:kafka:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.kafka.KafkaIOIT.testKafkaIOReadsAndWritesCorrectlyInBatch \ + --info \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{env.beam_PerformanceTests_Kafka_IO_test_arguments_2}}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt b/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt new file mode 100644 index 000000000000..bad323daacd5 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt @@ -0,0 +1,26 @@ +# 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. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--sourceOptions={\"numRecords\":\"100000000\",\"keySizeBytes\":\"10\",\"valueSizeBytes\":\"90\"} +--bigQueryDataset=beam_performance +--bigQueryTable=kafkaioit_results +--influxMeasurement=kafkaioit_results +--kafkaTopic=beam-batch +--readTimeout=1800 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt b/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt new file mode 100644 index 000000000000..c44ce0180c2d --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--sourceOptions={\"numRecords\":\"100000000\",\"keySizeBytes\":\"10\",\"valueSizeBytes\":\"90\"} +--bigQueryDataset=beam_performance +--bigQueryTable=kafkaioit_results_runner_v2 +--influxMeasurement=kafkaioit_results_runner_v2 +--kafkaTopic=beam-sdf +--readTimeout=1500 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--experiments=use_runner_v2,use_unified_worker \ No newline at end of file From 1c0f1beede08699f3cb809bb4567ec9f332573b9 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Thu, 28 Sep 2023 21:25:54 +0600 Subject: [PATCH 18/64] Github Workflow Replacement for Jenkins Jobs, beam_PerformanceTests_XmlIOIT* and beam_PerformanceTests_TFRecordIOIT (#28584) * beam_PerformanceTests_XmlIOIT_and_TFRecordIOIT * fix * cron and whitespace fix --- .../beam_PerformanceTests_TFRecordIOIT.yml | 91 +++++++++++++++ .../beam_PerformanceTests_XmlIOIT.yml | 91 +++++++++++++++ .../beam_PerformanceTests_XmlIOIT_HDFS.yml | 109 ++++++++++++++++++ .../config_TFRecordIOIT.txt | 26 +++++ .../config_XmlIOIT.txt | 27 +++++ .../config_XmlIOIT_HDFS.txt | 27 +++++ 6 files changed, 371 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_XmlIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml create mode 100644 .github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/config_XmlIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml new file mode 100644 index 000000000000..aa96b7e2bf81 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests TFRecordIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 4/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_TFRecordIOIT_test_arguments_1: '' + +jobs: + beam_PerformanceTests_TFRecordIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java TFRecordIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_TFRecordIOIT"] + job_phrase: ["Run Java TFRecordIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.tfrecord.TFRecordIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_TFRecordIOIT_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml new file mode 100644 index 000000000000..cd0245f269d1 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests XmlIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 4/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_XmlIOIT_test_arguments_1: '' + +jobs: + beam_PerformanceTests_XmlIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java XmlIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_XmlIOIT"] + job_phrase: ["Run Java XmlIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.xml.XmlIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_XmlIOIT_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml new file mode 100644 index 000000000000..a89f0b5dcb0f --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -0,0 +1,109 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PerformanceTests XmlIOIT HDFS + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 4/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_XmlIOIT_HDFS_test_arguments_1: '' + +jobs: + beam_PerformanceTests_XmlIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java XmlIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_XmlIOIT_HDFS"] + job_phrase: ["Run Java XmlIO Performance Test HDFS"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.xml.XmlIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_XmlIOIT_HDFS_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt b/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt new file mode 100644 index 000000000000..bb263609019a --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt @@ -0,0 +1,26 @@ +# 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. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=tfrecordioit_results +--influxMeasurement=tfrecordioit_results +--numberOfRecords=18000000 +--expectedHash=543104423f8b6eb097acb9f111c19fe4 +--datasetSize=1019380000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt b/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt new file mode 100644 index 000000000000..5847dc34aa58 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=xmlioit_results +--influxMeasurement=xmlioit_results +--numberOfRecords=12000000 +--expectedHash=b3b717e7df8f4878301b20f314512fb3 +--datasetSize=1076590000 +--charset=UTF-8 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt new file mode 100644 index 000000000000..0de45091d484 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=xmlioit_hdfs_results +--influxMeasurement=xmlioit_hdfs_results +--numberOfRecords=12000000 +--expectedHash=b3b717e7df8f4878301b20f314512fb3 +--datasetSize=1076590000 +--charset=UTF-8 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file From e328ab51f1b8fcb035af7257d93692ed28f4d7ea Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Thu, 28 Sep 2023 21:28:07 +0600 Subject: [PATCH 19/64] Github Workflow Replacement for Jenkins Jobs, beam_PerformanceTests_ManyFiles_TextIOIT* (#28581) * beam_PerformanceTests_ManyFiles_TextIOIT * fix * cron fix and whitespace --- ...am_PerformanceTests_ManyFiles_TextIOIT.yml | 91 +++++++++++++++ ...rformanceTests_ManyFiles_TextIOIT_HDFS.yml | 109 ++++++++++++++++++ .../config_ManyFiles_TextIOIT.txt | 29 +++++ .../config_ManyFiles_TextIOIT_HDFS.txt | 29 +++++ 4 files changed, 258 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml create mode 100644 .github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml new file mode 100644 index 000000000000..5834bd8ab3e0 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests ManyFiles TextIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 2/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_ManyFiles_TextIOIT_test_arguments_1: '' + +jobs: + beam_PerformanceTests_ManyFiles_TextIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java ManyFilesTextIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_ManyFiles_TextIOIT"] + job_phrase: ["Run Java ManyFilesTextIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_ManyFiles_TextIOIT_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml new file mode 100644 index 000000000000..03163a41dcf9 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -0,0 +1,109 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PerformanceTests ManyFiles TextIOIT HDFS + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 2/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + beam_PerformanceTests_ManyFiles_TextIOIT_HDFS_test_arguments_1: '' + +jobs: + beam_PerformanceTests_ManyFiles_TextIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java ManyFilesTextIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_ManyFiles_TextIOIT_HDFS"] + job_phrase: ["Run Java ManyFilesTextIO Performance Test HDFS"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_ManyFiles_TextIOIT_HDFS_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt b/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt new file mode 100644 index 000000000000..dcb6e31526ff --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=many_files_textioit_results +--influxMeasurement=many_files_textioit_results +--reportGcsPerformanceMetrics=true +--gcsPerformanceMetrics=true +--numberOfRecords=25000000 +--expectedHash=f8453256ccf861e8a312c125dfe0e436 +--datasetSize=1062290000 +--numberOfShards=1000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt new file mode 100644 index 000000000000..f01a4f488312 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=many_files_textioit_hdfs_results +--influxMeasurement=many_files_textioit_hdfs_results +--reportGcsPerformanceMetrics=true +--gcsPerformanceMetrics=true +--numberOfRecords=25000000 +--expectedHash=f8453256ccf861e8a312c125dfe0e436 +--datasetSize=1062290000 +--numberOfShards=1000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file From dda0eb9d642a8f3b23174a6c39a587c10726f30f Mon Sep 17 00:00:00 2001 From: Damon Date: Thu, 28 Sep 2023 09:08:37 -0700 Subject: [PATCH 20/64] Create RequestResponseIO gradle project (#28706) --- sdks/java/io/rrio/build.gradle | 36 ++++++++++++++++++++++++++++++++++ settings.gradle.kts | 1 + 2 files changed, 37 insertions(+) create mode 100644 sdks/java/io/rrio/build.gradle diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle new file mode 100644 index 000000000000..d65df370e0ca --- /dev/null +++ b/sdks/java/io/rrio/build.gradle @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.rrio' +) + +description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" +ext.summary = "Support to read from and write to Web APIs" + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.joda_time + implementation library.java.vendored_guava_32_1_2_jre + + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.junit + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testRuntimeOnly library.java.slf4j_jdk14 +} \ No newline at end of file diff --git a/settings.gradle.kts b/settings.gradle.kts index 45b8c25101b5..f4901d7df92b 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -255,6 +255,7 @@ include(":sdks:java:io:parquet") include(":sdks:java:io:pulsar") include(":sdks:java:io:rabbitmq") include(":sdks:java:io:redis") +include(":sdks:java:io:rrio") include(":sdks:java:io:solr") include(":sdks:java:io:sparkreceiver:2") include(":sdks:java:io:snowflake") From b9ac87ecbe2593276b50427295e2147c93240bf9 Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Thu, 28 Sep 2023 18:36:57 +0200 Subject: [PATCH 21/64] add write to checks due to publish unit test result (#28720) * add write to checks due to publish unit test result * add pull_request write due to comment mode --- .github/workflows/beam_PostCommit_Java.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 3e3cd79e8477..d2d3be47e45d 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -32,8 +32,8 @@ concurrency: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none From b371ab9e0b33fdd0bd4a5886d59cab5671287303 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 28 Sep 2023 12:47:08 -0400 Subject: [PATCH 22/64] Bump get-func-name from 2.0.0 to 2.0.2 in /sdks/typescript (#28707) Bumps [get-func-name](https://github.com/chaijs/get-func-name) from 2.0.0 to 2.0.2. - [Release notes](https://github.com/chaijs/get-func-name/releases) - [Commits](https://github.com/chaijs/get-func-name/commits/v2.0.2) --- updated-dependencies: - dependency-name: get-func-name dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/typescript/package-lock.json | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/sdks/typescript/package-lock.json b/sdks/typescript/package-lock.json index e4556449fde4..55d510ccce8c 100644 --- a/sdks/typescript/package-lock.json +++ b/sdks/typescript/package-lock.json @@ -1,15 +1,15 @@ { "name": "apache-beam", - "version": "2.50.0", + "version": "2.52.0-SNAPSHOT", "lockfileVersion": 2, "requires": true, "packages": { "": { "name": "apache-beam", - "version": "2.50.0", + "version": "2.52.0-SNAPSHOT", "dependencies": { "@google-cloud/pubsub": "^2.19.4", - "@grpc/grpc-js": "^1.4.6", + "@grpc/grpc-js": "~1.4.6", "@protobuf-ts/grpc-transport": "^2.1.0", "@protobuf-ts/plugin": "^2.1.0", "bson": "^4.6.0", @@ -19,7 +19,7 @@ "fast-deep-equal": "^3.1.3", "find-git-root": "^1.0.4", "long": "^4.0.0", - "protobufjs": "^6.11.3", + "protobufjs": "~6.11.3", "queue-typescript": "^1.0.1", "serialize-closures": "^0.2.7", "ts-closure-transform": "^0.1.7", @@ -1972,9 +1972,9 @@ } }, "node_modules/get-func-name": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/get-func-name/-/get-func-name-2.0.0.tgz", - "integrity": "sha1-6td0q+5y4gQJQzoGY2YCPdaIekE=", + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/get-func-name/-/get-func-name-2.0.2.tgz", + "integrity": "sha512-8vXOvuE167CtIc3OyItco7N/dpRtBbYOsPsXCz7X/PMnlGjYjSGuZJgM1Y7mmew7BKf9BqvLX2tnOVy1BBUsxQ==", "engines": { "node": "*" } @@ -5495,9 +5495,9 @@ "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==" }, "get-func-name": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/get-func-name/-/get-func-name-2.0.0.tgz", - "integrity": "sha1-6td0q+5y4gQJQzoGY2YCPdaIekE=" + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/get-func-name/-/get-func-name-2.0.2.tgz", + "integrity": "sha512-8vXOvuE167CtIc3OyItco7N/dpRtBbYOsPsXCz7X/PMnlGjYjSGuZJgM1Y7mmew7BKf9BqvLX2tnOVy1BBUsxQ==" }, "glob": { "version": "7.2.0", From 5f6817e8213a0e8d5e4caa6472114e35f6bf678f Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 28 Sep 2023 13:13:23 -0400 Subject: [PATCH 23/64] Run other arm tests on Dataflow Java (#28666) * Run other arm tests on Dataflow Java * Set max workers --- .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 1 + runners/google-cloud-dataflow-java/arm/build.gradle | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 5a42b9f95237..22d605ed1c2d 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -109,6 +109,7 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :runners:google-cloud-dataflow-java:arm:examplesJavaRunnerV2IntegrationTestARM + max-workers: 12 arguments: | -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index 10de65199259..e79eeedcd828 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -128,8 +128,9 @@ task examplesJavaRunnerV2IntegrationTestARM(type: Test) { systemProperty "beamTestPipelineOptions", JsonOutput.toJson(runnerV2PipelineOptionsARM) - include '**/WordCountIT.class' - include '**/WindowedWordCountIT.class' + include '**/*IT.class' + // TODO(https://github.com/apache/beam/issues/20593): test times out. + exclude '**/FhirIOReadIT.class' maxParallelForks 4 classpath = configurations.examplesJavaIntegrationTest From 2b7df3b64b3ee2487ac9f6ee53cff468ba6c516e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 28 Sep 2023 13:28:51 -0400 Subject: [PATCH 24/64] Bump github.com/aws/aws-sdk-go-v2/service/s3 in /sdks (#28674) Bumps [github.com/aws/aws-sdk-go-v2/service/s3](https://github.com/aws/aws-sdk-go-v2) from 1.39.0 to 1.40.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/service/s3/v1.39.0...service/s3/v1.40.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/service/s3 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index f8536a19d155..e1ad4a9707e4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -34,7 +34,7 @@ require ( github.com/aws/aws-sdk-go-v2/config v1.18.42 github.com/aws/aws-sdk-go-v2/credentials v1.13.40 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.86 - github.com/aws/aws-sdk-go-v2/service/s3 v1.39.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 github.com/aws/smithy-go v1.14.2 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 diff --git a/sdks/go.sum b/sdks/go.sum index 4559cdbf4d9c..d72d5ab27ba3 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -118,8 +118,9 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNN github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 h1:v0jkRigbSD6uOdwcaUQmgEwG1BkPfAPDqaeNt/29ghg= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4/go.mod h1:LhTyt8J04LL+9cIt7pYJ5lbS/U98ZmXovLOR/4LUsk8= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.39.0 h1:VZ2WMkKLio5tVjYfThcy5+pb6YHGd6B6egq75FfM6hU= github.com/aws/aws-sdk-go-v2/service/s3 v1.39.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= github.com/aws/aws-sdk-go-v2/service/sso v1.14.1 h1:YkNzx1RLS0F5qdf9v1Q8Cuv9NXCL2TkosOxhzlUPV64= github.com/aws/aws-sdk-go-v2/service/sso v1.14.1/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= From e6dcd372123c42226b1384704cc1d2baac35f3f1 Mon Sep 17 00:00:00 2001 From: Israel Herraiz Date: Thu, 28 Sep 2023 19:30:57 +0200 Subject: [PATCH 25/64] Update example in Tour of Beam (#28718) This example was mixing the with-statement style of doing pipelines with actually calling the run method. This changes the example to only use the with-statement, which is the most common style found in the documentation. --- .../setting-pipeline/python-example/task.py | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/learning/tour-of-beam/learning-content/introduction/introduction-concepts/pipeline-concepts/setting-pipeline/python-example/task.py b/learning/tour-of-beam/learning-content/introduction/introduction-concepts/pipeline-concepts/setting-pipeline/python-example/task.py index 91ce88c6fa5e..29fc5997b4c7 100644 --- a/learning/tour-of-beam/learning-content/introduction/introduction-concepts/pipeline-concepts/setting-pipeline/python-example/task.py +++ b/learning/tour-of-beam/learning-content/introduction/introduction-concepts/pipeline-concepts/setting-pipeline/python-example/task.py @@ -69,18 +69,13 @@ def main(argv=None, save_main_session=True): pipeline_options.view_as(SetupOptions).save_main_session = save_main_session with beam.Pipeline(options=pipeline_options) as p: - - # Read the text file[pattern] into a PCollection. - lines = p | 'Read' >> ReadFromText(known_args.input) \ + # Read the text file[pattern] into a PCollection. + lines = p | 'Read' >> ReadFromText(known_args.input) \ | beam.Filter(lambda line: line != "") - - # Write the output using a "Write" transform that has side effects. - # pylint: disable=expression-not-assigned - output = lines | 'Write' >> WriteToText(known_args.output) - - - result = p.run() - result.wait_until_finish() + + # Write the output using a "Write" transform that has side effects. + # pylint: disable=expression-not-assigned + output = lines | 'Write' >> WriteToText(known_args.output) if __name__ == '__main__': From 56b1259cac9fef5012b40ea6173e01269909c257 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 28 Sep 2023 13:34:27 -0400 Subject: [PATCH 26/64] Revert "Replace StorageV1 client with GCS client (#28079)" (#28721) This reverts commit 94812c0a272b65246f4725c1a4ed78dcbd6a619c. --- CHANGES.md | 6 - .../assets/symbols/python.g.yaml | 8 + .../examples/complete/game/user_score.py | 1 - sdks/python/apache_beam/internal/gcp/auth.py | 7 +- .../apache_beam/io/gcp/bigquery_test.py | 4 +- .../apache_beam/io/gcp/bigquery_tools.py | 4 - .../apache_beam/io/gcp/bigquery_tools_test.py | 5 +- .../apache_beam/io/gcp/gcsfilesystem.py | 37 +- .../io/gcp/gcsfilesystem_integration_test.py | 9 +- .../apache_beam/io/gcp/gcsfilesystem_test.py | 74 +- sdks/python/apache_beam/io/gcp/gcsio.py | 641 +++- .../io/gcp/gcsio_integration_test.py | 188 +- .../apache_beam/io/gcp/gcsio_overrides.py | 55 + sdks/python/apache_beam/io/gcp/gcsio_test.py | 886 ++++-- .../gcp/internal/clients/storage/__init__.py | 33 + .../clients/storage/storage_v1_client.py | 1517 +++++++++ .../clients/storage/storage_v1_messages.py | 2714 +++++++++++++++++ .../pipeline_options_validator_test.py | 1 - .../runners/dataflow/internal/apiclient.py | 60 +- .../apache_beam/runners/interactive/utils.py | 26 +- .../runners/interactive/utils_test.py | 41 +- .../portability/sdk_container_builder.py | 41 +- sdks/python/mypy.ini | 3 + sdks/python/setup.py | 1 - 24 files changed, 5752 insertions(+), 610 deletions(-) create mode 100644 sdks/python/apache_beam/io/gcp/gcsio_overrides.py create mode 100644 sdks/python/apache_beam/io/gcp/internal/clients/storage/__init__.py create mode 100644 sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py create mode 100644 sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py diff --git a/CHANGES.md b/CHANGES.md index a9c96bfb81f0..650b33c12407 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -53,12 +53,6 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> -# [2.52.0] - Unreleased - -## I/Os - -* Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) - # [2.51.0] - Unreleased ## Highlights diff --git a/playground/frontend/playground_components/assets/symbols/python.g.yaml b/playground/frontend/playground_components/assets/symbols/python.g.yaml index 0b9e5e142ded..a47447225a68 100644 --- a/playground/frontend/playground_components/assets/symbols/python.g.yaml +++ b/playground/frontend/playground_components/assets/symbols/python.g.yaml @@ -4790,6 +4790,10 @@ GBKTransform: - from_runner_api_parameter - to_runner_api_parameter GcpTestIOError: {} +GcsDownloader: + methods: + - get_range + - size GCSFileSystem: methods: - checksum @@ -4833,6 +4837,10 @@ GcsIOError: {} GcsIOOverrides: methods: - retry_func +GcsUploader: + methods: + - finish + - put GeneralPurposeConsumerSet: methods: - flush diff --git a/sdks/python/apache_beam/examples/complete/game/user_score.py b/sdks/python/apache_beam/examples/complete/game/user_score.py index 03f0d00fc30f..564cea8c425c 100644 --- a/sdks/python/apache_beam/examples/complete/game/user_score.py +++ b/sdks/python/apache_beam/examples/complete/game/user_score.py @@ -177,7 +177,6 @@ def format_user_score_sums(user_score): (user, score) = user_score return 'user: %s, total_score: %s' % (user, score) - ( # pylint: disable=expression-not-assigned p | 'ReadInputText' >> beam.io.ReadFromText(args.input) diff --git a/sdks/python/apache_beam/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py index b2fda2c6e897..bab3ace4144e 100644 --- a/sdks/python/apache_beam/internal/gcp/auth.py +++ b/sdks/python/apache_beam/internal/gcp/auth.py @@ -112,9 +112,6 @@ def __getattr__(self, attr): """Delegate attribute access to underlying google-auth credentials.""" return getattr(self._google_auth_credentials, attr) - def get_google_auth_credentials(self): - return self._google_auth_credentials - class _Credentials(object): _credentials_lock = threading.Lock() @@ -123,7 +120,7 @@ class _Credentials(object): @classmethod def get_service_credentials(cls, pipeline_options): - # type: (PipelineOptions) -> Optional[_ApitoolsCredentialsAdapter] + # type: (PipelineOptions) -> Optional[google.auth.credentials.Credentials] with cls._credentials_lock: if cls._credentials_init: return cls._credentials @@ -143,7 +140,7 @@ def get_service_credentials(cls, pipeline_options): @staticmethod def _get_service_credentials(pipeline_options): - # type: (PipelineOptions) -> Optional[_ApitoolsCredentialsAdapter] + # type: (PipelineOptions) -> Optional[google.auth.credentials.Credentials] if not _GOOGLE_AUTH_AVAILABLE: _LOGGER.warning( 'Unable to find default credentials because the google-auth library ' diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 41ad6fc146c8..7e9c1e634748 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -826,7 +826,6 @@ def test_streaming_inserts_flush_on_byte_size_limit(self, mock_insert): exception_type=exceptions.ServiceUnavailable if exceptions else None, error_message='backendError') ]) - @unittest.skip('Not compatible with new GCS client. See GH issue #26334.') def test_load_job_exception(self, exception_type, error_message): with mock.patch.object(bigquery_v2_client.BigqueryV2.JobsService, @@ -866,7 +865,6 @@ def test_load_job_exception(self, exception_type, error_message): exception_type=exceptions.InternalServerError if exceptions else None, error_message='internalError'), ]) - @unittest.skip('Not compatible with new GCS client. See GH issue #26334.') def test_copy_load_job_exception(self, exception_type, error_message): from apache_beam.io.gcp import bigquery_file_loads @@ -885,7 +883,7 @@ def test_copy_load_job_exception(self, exception_type, error_message): mock.patch.object(BigQueryWrapper, 'wait_for_bq_job'), \ mock.patch('apache_beam.io.gcp.internal.clients' - '.storage.storage_v1_client.StorageV1.ObjectsService'),\ + '.storage.storage_v1_client.StorageV1.ObjectsService'), \ mock.patch('time.sleep'), \ self.assertRaises(Exception) as exc, \ beam.Pipeline() as p: diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index c2a1eba8b983..2f9420795288 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -339,9 +339,6 @@ class BigQueryWrapper(object): offer a common place where retry logic for failures can be controlled. In addition, it offers various functions used both in sources and sinks (e.g., find and create tables, query a table, etc.). - - Note that client parameter in constructor is only for testing purposes and - should not be used in production code. """ # If updating following names, also update the corresponding pydocs in @@ -356,7 +353,6 @@ def __init__(self, client=None, temp_dataset_id=None, temp_table_ref=None): self.gcp_bq_client = client or gcp_bigquery.Client( client_info=ClientInfo( user_agent="apache-beam-%s" % apache_beam.__version__)) - self._unique_row_id = 0 # For testing scenarios where we pass in a client we do not want a # randomized prefix for row IDs. diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py index b1e0b898558a..b4c84d589c07 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py @@ -60,6 +60,7 @@ from apitools.base.py.exceptions import HttpError, HttpForbiddenError from google.api_core.exceptions import ClientError, DeadlineExceeded from google.api_core.exceptions import InternalServerError + import google.cloud except ImportError: ClientError = None DeadlineExceeded = None @@ -223,9 +224,11 @@ def test_delete_dataset_retries_for_timeouts(self, patched_time_sleep): wrapper._delete_dataset('', '') self.assertTrue(client.datasets.Delete.called) + @unittest.skipIf( + google and not hasattr(google.cloud, '_http'), # pylint: disable=c-extension-no-member + 'Dependencies not installed') @mock.patch('time.sleep', return_value=None) @mock.patch('google.cloud._http.JSONConnection.http') - @unittest.skip('Fails on import') def test_user_agent_insert_all(self, http_mock, patched_sleep): wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper() try: diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py index 173b21a38f88..c87a8499c91d 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py @@ -257,18 +257,17 @@ def rename(self, source_file_names, destination_file_names): exceptions = {} for batch in gcs_batches: copy_statuses = self._gcsIO().copy_batch(batch) - copy_succeeded = {} - delete_targets = [] + copy_succeeded = [] for src, dest, exception in copy_statuses: if exception: exceptions[(src, dest)] = exception else: - copy_succeeded[src] = dest - delete_targets.append(src) - delete_statuses = self._gcsIO().delete_batch(delete_targets) - for src, exception in delete_statuses: + copy_succeeded.append((src, dest)) + delete_batch = [src for src, dest in copy_succeeded] + delete_statuses = self._gcsIO().delete_batch(delete_batch) + for i, (src, exception) in enumerate(delete_statuses): + dest = copy_succeeded[i][1] if exception: - dest = copy_succeeded[src] exceptions[(src, dest)] = exception if exceptions: @@ -341,7 +340,8 @@ def metadata(self, path): """ try: file_metadata = self._gcsIO()._status(path) - return FileMetadata(path, file_metadata['size'], file_metadata['updated']) + return FileMetadata( + path, file_metadata['size'], file_metadata['last_updated']) except Exception as e: # pylint: disable=broad-except raise BeamIOError("Metadata operation failed", {path: e}) @@ -352,10 +352,9 @@ def delete(self, paths): Args: paths: list of paths that give the file objects to be deleted """ - - exceptions = {} - - for path in paths: + def _delete_path(path): + """Recursively delete the file or directory at the provided path. + """ if path.endswith('/'): path_to_use = path + '*' else: @@ -363,9 +362,17 @@ def delete(self, paths): match_result = self.match([path_to_use])[0] statuses = self._gcsIO().delete_batch( [m.path for m in match_result.metadata_list]) - for target, exception in statuses: - if exception: - exceptions[target] = exception + # pylint: disable=used-before-assignment + failures = [e for (_, e) in statuses if e is not None] + if failures: + raise failures[0] + + exceptions = {} + for path in paths: + try: + _delete_path(path) + except Exception as e: # pylint: disable=broad-except + exceptions[path] = e if exceptions: raise BeamIOError("Delete operation failed", exceptions) diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem_integration_test.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem_integration_test.py index 3cefb6ca30d8..c322e4c9e1cc 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem_integration_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem_integration_test.py @@ -21,9 +21,9 @@ Instantiates a TestPipeline to get options such as GCP project name, but doesn't actually start a Beam pipeline or test any specific runner. -Run the following in 'sdks/python' directory to run these tests manually: - scripts/run_integration_test.sh \ - --test_opts apache_beam/io/gcp/gcsfilesystem_integration_test.py +To run these tests manually: + ./gradlew :sdks:python:test-suites:dataflow:integrationTest \ + -Dtests=apache_beam.io.gcp.gcsfilesystem_integration_test:GcsFileSystemIntegrationTest # pylint: disable=line-too-long """ # pytype: skip-file @@ -49,6 +49,9 @@ class GcsFileSystemIntegrationTest(unittest.TestCase): INPUT_FILE = 'gs://dataflow-samples/shakespeare/kinglear.txt' + # Larger than 1MB to test maxBytesRewrittenPerCall. + # Also needs to be in a different region than the dest to take effect. + INPUT_FILE_LARGE = 'gs://apache-beam-samples-us-east1/wikipedia_edits/wiki_data-000000000000.json' # pylint: disable=line-too-long def setUp(self): self.test_pipeline = TestPipeline(is_integration_test=True) diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py index 1206529faf01..800bd5d1c462 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py @@ -258,7 +258,7 @@ def test_rename(self, mock_gcsio): ]) @mock.patch('apache_beam.io.gcp.gcsfilesystem.gcsio') - def test_rename_delete_error(self, mock_gcsio): + def test_rename_error(self, mock_gcsio): # Prepare mocks. gcsio_mock = mock.MagicMock() gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock @@ -272,60 +272,25 @@ def test_rename_delete_error(self, mock_gcsio): 'gs://bucket/to2', 'gs://bucket/to3', ] + exception = IOError('Failed') + gcsio_mock.delete_batch.side_effect = [[(f, exception) for f in sources]] gcsio_mock.copy_batch.side_effect = [[ ('gs://bucket/from1', 'gs://bucket/to1', None), ('gs://bucket/from2', 'gs://bucket/to2', None), ('gs://bucket/from3', 'gs://bucket/to3', None), ]] - gcsio_mock.delete_batch.side_effect = [[ - ('gs://bucket/from1', None), - ('gs://bucket/from2', Exception("BadThings")), - ('gs://bucket/from3', None), - ]] # Issue batch rename. - with self.assertRaisesRegex(BeamIOError, r'^Rename operation failed'): - self.fs.rename(sources, destinations) - - gcsio_mock.copy_batch.assert_called_once_with([ - ('gs://bucket/from1', 'gs://bucket/to1'), - ('gs://bucket/from2', 'gs://bucket/to2'), - ('gs://bucket/from3', 'gs://bucket/to3'), - ]) - gcsio_mock.delete_batch.assert_called_once_with([ - 'gs://bucket/from1', - 'gs://bucket/from2', - 'gs://bucket/from3', - ]) - - @mock.patch('apache_beam.io.gcp.gcsfilesystem.gcsio') - def test_rename_copy_error(self, mock_gcsio): - # Prepare mocks. - gcsio_mock = mock.MagicMock() - gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock - sources = [ - 'gs://bucket/from1', - 'gs://bucket/from2', - 'gs://bucket/from3', - ] - destinations = [ - 'gs://bucket/to1', - 'gs://bucket/to2', - 'gs://bucket/to3', - ] - gcsio_mock.copy_batch.side_effect = [[ - ('gs://bucket/from1', 'gs://bucket/to1', None), - ('gs://bucket/from2', 'gs://bucket/to2', Exception("BadThings")), - ('gs://bucket/from3', 'gs://bucket/to3', None), - ]] - gcsio_mock.delete_batch.side_effect = [[ - ('gs://bucket/from1', None), - ('gs://bucket/from3', None), - ]] + expected_results = { + (s, d): exception + for s, d in zip(sources, destinations) + } # Issue batch rename. - with self.assertRaisesRegex(BeamIOError, r'^Rename operation failed'): + with self.assertRaisesRegex(BeamIOError, + r'^Rename operation failed') as error: self.fs.rename(sources, destinations) + self.assertEqual(error.exception.exception_details, expected_results) gcsio_mock.copy_batch.assert_called_once_with([ ('gs://bucket/from1', 'gs://bucket/to1'), @@ -334,6 +299,7 @@ def test_rename_copy_error(self, mock_gcsio): ]) gcsio_mock.delete_batch.assert_called_once_with([ 'gs://bucket/from1', + 'gs://bucket/from2', 'gs://bucket/from3', ]) @@ -342,7 +308,7 @@ def test_delete(self, mock_gcsio): # Prepare mocks. gcsio_mock = mock.MagicMock() gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock - gcsio_mock._status.return_value = {'size': 0, 'updated': 99999.0} + gcsio_mock._status.return_value = {'size': 0, 'last_updated': 99999.0} files = [ 'gs://bucket/from1', 'gs://bucket/from2', @@ -358,21 +324,21 @@ def test_delete_error(self, mock_gcsio): # Prepare mocks. gcsio_mock = mock.MagicMock() gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock - - gcsio_mock._status.return_value = {'size': 0, 'updated': 99999.0} + exception = IOError('Failed') + gcsio_mock.delete_batch.side_effect = exception + gcsio_mock._status.return_value = {'size': 0, 'last_updated': 99999.0} files = [ 'gs://bucket/from1', 'gs://bucket/from2', 'gs://bucket/from3', ] - gcsio_mock.delete_batch.side_effect = [ - [('gs://bucket/from1', None)], - [('gs://bucket/from2', Exception("BadThings"))], - [('gs://bucket/from3', None)], - ] + expected_results = {f: exception for f in files} + # Issue batch delete. - with self.assertRaisesRegex(BeamIOError, r'^Delete operation failed'): + with self.assertRaisesRegex(BeamIOError, + r'^Delete operation failed') as error: self.fs.delete(files) + self.assertEqual(error.exception.exception_details, expected_results) gcsio_mock.delete_batch.assert_called() diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index e82b9b9a838e..d75af4fe6ac1 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -29,18 +29,28 @@ # pytype: skip-file +import errno +import io import logging +import multiprocessing import re +import threading import time +import traceback +from itertools import islice from typing import Optional from typing import Union -from google.cloud import storage -from google.cloud.exceptions import NotFound -from google.cloud.storage.fileio import BlobReader -from google.cloud.storage.fileio import BlobWriter - -from apache_beam.internal.gcp import auth +import apache_beam +from apache_beam.internal.http_client import get_new_http +from apache_beam.internal.metrics.metric import ServiceCallMetric +from apache_beam.io.filesystemio import Downloader +from apache_beam.io.filesystemio import DownloaderStream +from apache_beam.io.filesystemio import PipeStream +from apache_beam.io.filesystemio import Uploader +from apache_beam.io.filesystemio import UploaderStream +from apache_beam.io.gcp import resource_identifiers +from apache_beam.metrics import monitoring_infos from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.utils import retry from apache_beam.utils.annotations import deprecated @@ -49,11 +59,58 @@ _LOGGER = logging.getLogger(__name__) +# Issue a friendlier error message if the storage library is not available. +# TODO(silviuc): Remove this guard when storage is available everywhere. +try: + # pylint: disable=wrong-import-order, wrong-import-position + # pylint: disable=ungrouped-imports + from apitools.base.py.batch import BatchApiRequest + from apitools.base.py.exceptions import HttpError + from apitools.base.py import transfer + from apache_beam.internal.gcp import auth + from apache_beam.io.gcp.internal.clients import storage +except ImportError: + raise ImportError( + 'Google Cloud Storage I/O not supported for this execution environment ' + '(could not import storage API client).') + +# This is the size of each partial-file read operation from GCS. This +# parameter was chosen to give good throughput while keeping memory usage at +# a reasonable level; the following table shows throughput reached when +# reading files of a given size with a chosen buffer size and informed the +# choice of the value, as of 11/2016: +# +# +---------------+------------+-------------+-------------+-------------+ +# | | 50 MB file | 100 MB file | 200 MB file | 400 MB file | +# +---------------+------------+-------------+-------------+-------------+ +# | 8 MB buffer | 17.12 MB/s | 22.67 MB/s | 23.81 MB/s | 26.05 MB/s | +# | 16 MB buffer | 24.21 MB/s | 42.70 MB/s | 42.89 MB/s | 46.92 MB/s | +# | 32 MB buffer | 28.53 MB/s | 48.08 MB/s | 54.30 MB/s | 54.65 MB/s | +# | 400 MB buffer | 34.72 MB/s | 71.13 MB/s | 79.13 MB/s | 85.39 MB/s | +# +---------------+------------+-------------+-------------+-------------+ DEFAULT_READ_BUFFER_SIZE = 16 * 1024 * 1024 +# This is the number of seconds the library will wait for a partial-file read +# operation from GCS to complete before retrying. +DEFAULT_READ_SEGMENT_TIMEOUT_SECONDS = 60 + +# This is the size of chunks used when writing to GCS. +WRITE_CHUNK_SIZE = 8 * 1024 * 1024 + # Maximum number of operations permitted in GcsIO.copy_batch() and # GcsIO.delete_batch(). -MAX_BATCH_OPERATION_SIZE = 1000 +MAX_BATCH_OPERATION_SIZE = 100 + +# Batch endpoint URL for GCS. +# We have to specify an API specific endpoint here since Google APIs global +# batch endpoints will be deprecated on 03/25/2019. +# See https://developers.googleblog.com/2018/03/discontinuing-support-for-json-rpc-and.html. # pylint: disable=line-too-long +# Currently apitools library uses a global batch endpoint by default: +# https://github.com/google/apitools/blob/master/apitools/base/py/batch.py#L152 +# TODO: remove this constant and it's usage after apitools move to using an API +# specific batch endpoint or after Beam gcsio module start using a GCS client +# library that does not use global batch endpoints. +GCS_BATCH_ENDPOINT = 'https://www.googleapis.com/batch/storage/v1' def parse_gcs_path(gcs_path, object_optional=False): @@ -97,21 +154,29 @@ def get_or_create_default_gcs_bucket(options): bucket_name, project, location=region) +class GcsIOError(IOError, retry.PermanentException): + """GCS IO error that should not be retried.""" + pass + + class GcsIO(object): """Google Cloud Storage I/O client.""" def __init__(self, storage_client=None, pipeline_options=None): - # type: (Optional[storage.Client], Optional[Union[dict, PipelineOptions]]) -> None + # type: (Optional[storage.StorageV1], Optional[Union[dict, PipelineOptions]]) -> None if storage_client is None: if not pipeline_options: pipeline_options = PipelineOptions() elif isinstance(pipeline_options, dict): pipeline_options = PipelineOptions.from_dictionary(pipeline_options) - credentials = auth.get_service_credentials(pipeline_options) - if credentials: - storage_client = storage.Client( - credentials=credentials.get_google_auth_credentials()) - else: - storage_client = storage.Client.create_anonymous_client() + storage_client = storage.StorageV1( + credentials=auth.get_service_credentials(pipeline_options), + get_credentials=False, + http=get_new_http(), + response_encoding='utf8', + additional_http_headers={ + "User-Agent": "apache-beam/%s (GPN:Beam)" % + apache_beam.__version__ + }) self.client = storage_client self._rewrite_cb = None self.bucket_to_project_number = {} @@ -121,30 +186,40 @@ def get_project_number(self, bucket): bucket_metadata = self.get_bucket(bucket_name=bucket) if bucket_metadata: self.bucket_to_project_number[bucket] = bucket_metadata.projectNumber + # else failed to load the bucket metadata due to HttpError return self.bucket_to_project_number.get(bucket, None) + def _set_rewrite_response_callback(self, callback): + """For testing purposes only. No backward compatibility guarantees. + + Args: + callback: A function that receives ``storage.RewriteResponse``. + """ + self._rewrite_cb = callback + def get_bucket(self, bucket_name): """Returns an object bucket from its name, or None if it does not exist.""" try: - return self.client.lookup_bucket(bucket_name) - except NotFound: + request = storage.StorageBucketsGetRequest(bucket=bucket_name) + return self.client.buckets.Get(request) + except HttpError: return None def create_bucket(self, bucket_name, project, kms_key=None, location=None): """Create and return a GCS bucket in a specific project.""" - + encryption = None + if kms_key: + encryption = storage.Bucket.EncryptionValue(kms_key) + + request = storage.StorageBucketsInsertRequest( + bucket=storage.Bucket( + name=bucket_name, location=location, encryption=encryption), + project=project, + ) try: - bucket = self.client.create_bucket( - bucket_or_name=bucket_name, - project=project, - location=location, - ) - if kms_key: - bucket.default_kms_key_name(kms_key) - bucket.patch() - return bucket - except NotFound: + return self.client.buckets.Insert(request) + except HttpError: return None def open( @@ -167,18 +242,24 @@ def open( Raises: ValueError: Invalid open file mode. """ - bucket_name, blob_name = parse_gcs_path(filename) - bucket = self.client.get_bucket(bucket_name) - if mode == 'r' or mode == 'rb': - blob = bucket.get_blob(blob_name) - return BeamBlobReader(blob, chunk_size=read_buffer_size) + downloader = GcsDownloader( + self.client, + filename, + buffer_size=read_buffer_size, + get_project_number=self.get_project_number) + return io.BufferedReader( + DownloaderStream( + downloader, read_buffer_size=read_buffer_size, mode=mode), + buffer_size=read_buffer_size) elif mode == 'w' or mode == 'wb': - blob = bucket.get_blob(blob_name) - if not blob: - blob = storage.Blob(blob_name, bucket) - return BeamBlobWriter(blob, mime_type) - + uploader = GcsUploader( + self.client, + filename, + mime_type, + get_project_number=self.get_project_number) + return io.BufferedWriter( + UploaderStream(uploader, mode=mode), buffer_size=128 * 1024) else: raise ValueError('Invalid file open mode: %s.' % mode) @@ -190,13 +271,19 @@ def delete(self, path): Args: path: GCS file path pattern in the form gs:///. """ - bucket_name, blob_name = parse_gcs_path(path) + bucket, object_path = parse_gcs_path(path) + request = storage.StorageObjectsDeleteRequest( + bucket=bucket, object=object_path) try: - bucket = self.client.get_bucket(bucket_name) - bucket.delete_blob(blob_name) - except NotFound: - return - + self.client.objects.Delete(request) + except HttpError as http_error: + if http_error.status_code == 404: + # Return success when the file doesn't exist anymore for idempotency. + return + raise + + # We intentionally do not decorate this method with a retry, as retrying is + # handled in BatchApiRequest.Execute(). def delete_batch(self, paths): """Deletes the objects at the given GCS paths. @@ -204,105 +291,166 @@ def delete_batch(self, paths): paths: List of GCS file path patterns in the form gs:///, not to exceed MAX_BATCH_OPERATION_SIZE in length. - Returns: List of tuples of (path, exception) in the same order as the - paths argument, where exception is None if the operation - succeeded or the relevant exception if the operation failed. + Returns: List of tuples of (path, exception) in the same order as the paths + argument, where exception is None if the operation succeeded or + the relevant exception if the operation failed. """ - final_results = [] - s = 0 - while s < len(paths): - if (s + MAX_BATCH_OPERATION_SIZE) < len(paths): - current_paths = paths[s:s + MAX_BATCH_OPERATION_SIZE] - else: - current_paths = paths[s:] - current_batch = self.client.batch(raise_exception=False) - with current_batch: - for path in current_paths: - bucket_name, blob_name = parse_gcs_path(path) - bucket = self.client.get_bucket(bucket_name) - bucket.delete_blob(blob_name) - - for i, path in enumerate(current_paths): - error_code = None - for j in range(2): - resp = current_batch._responses[2 * i + j] - if resp.status_code >= 400 and resp.status_code != 404: - error_code = resp.status_code - break - final_results.append((path, error_code)) - - s += MAX_BATCH_OPERATION_SIZE - - return final_results + if not paths: + return [] + + paths = iter(paths) + result_statuses = [] + while True: + paths_chunk = list(islice(paths, MAX_BATCH_OPERATION_SIZE)) + if not paths_chunk: + return result_statuses + batch_request = BatchApiRequest( + batch_url=GCS_BATCH_ENDPOINT, + retryable_codes=retry.SERVER_ERROR_OR_TIMEOUT_CODES, + response_encoding='utf-8') + for path in paths_chunk: + bucket, object_path = parse_gcs_path(path) + request = storage.StorageObjectsDeleteRequest( + bucket=bucket, object=object_path) + batch_request.Add(self.client.objects, 'Delete', request) + api_calls = batch_request.Execute(self.client._http) # pylint: disable=protected-access + for i, api_call in enumerate(api_calls): + path = paths_chunk[i] + exception = None + if api_call.is_error: + exception = api_call.exception + # Return success when the file doesn't exist anymore for idempotency. + if isinstance(exception, HttpError) and exception.status_code == 404: + exception = None + result_statuses.append((path, exception)) + return result_statuses @retry.with_exponential_backoff( retry_filter=retry.retry_on_server_errors_and_timeout_filter) - def copy(self, src, dest): + def copy( + self, + src, + dest, + dest_kms_key_name=None, + max_bytes_rewritten_per_call=None): """Copies the given GCS object from src to dest. Args: src: GCS file path pattern in the form gs:///. dest: GCS file path pattern in the form gs:///. + dest_kms_key_name: Experimental. No backwards compatibility guarantees. + Encrypt dest with this Cloud KMS key. If None, will use dest bucket + encryption defaults. + max_bytes_rewritten_per_call: Experimental. No backwards compatibility + guarantees. Each rewrite API call will return after these many bytes. + Used for testing. Raises: TimeoutError: on timeout. """ - src_bucket_name, src_blob_name = parse_gcs_path(src) - dest_bucket_name, dest_blob_name= parse_gcs_path(dest, object_optional=True) - src_bucket = self.get_bucket(src_bucket_name) - src_blob = src_bucket.get_blob(src_blob_name) - if not src_blob: - raise NotFound("Source %s not found", src) - dest_bucket = self.get_bucket(dest_bucket_name) - if not dest_blob_name: - dest_blob_name = None - src_bucket.copy_blob(src_blob, dest_bucket, new_name=dest_blob_name) - - def copy_batch(self, src_dest_pairs): - """Copies the given GCS objects from src to dest. + src_bucket, src_path = parse_gcs_path(src) + dest_bucket, dest_path = parse_gcs_path(dest) + request = storage.StorageObjectsRewriteRequest( + sourceBucket=src_bucket, + sourceObject=src_path, + destinationBucket=dest_bucket, + destinationObject=dest_path, + destinationKmsKeyName=dest_kms_key_name, + maxBytesRewrittenPerCall=max_bytes_rewritten_per_call) + response = self.client.objects.Rewrite(request) + while not response.done: + _LOGGER.debug( + 'Rewrite progress: %d of %d bytes, %s to %s', + response.totalBytesRewritten, + response.objectSize, + src, + dest) + request.rewriteToken = response.rewriteToken + response = self.client.objects.Rewrite(request) + if self._rewrite_cb is not None: + self._rewrite_cb(response) + + _LOGGER.debug('Rewrite done: %s to %s', src, dest) + + # We intentionally do not decorate this method with a retry, as retrying is + # handled in BatchApiRequest.Execute(). + def copy_batch( + self, + src_dest_pairs, + dest_kms_key_name=None, + max_bytes_rewritten_per_call=None): + """Copies the given GCS object from src to dest. Args: src_dest_pairs: list of (src, dest) tuples of gs:/// files paths to copy from src to dest, not to exceed MAX_BATCH_OPERATION_SIZE in length. + dest_kms_key_name: Experimental. No backwards compatibility guarantees. + Encrypt dest with this Cloud KMS key. If None, will use dest bucket + encryption defaults. + max_bytes_rewritten_per_call: Experimental. No backwards compatibility + guarantees. Each rewrite call will return after these many bytes. Used + primarily for testing. Returns: List of tuples of (src, dest, exception) in the same order as the src_dest_pairs argument, where exception is None if the operation succeeded or the relevant exception if the operation failed. """ - final_results = [] - s = 0 - while s < len(src_dest_pairs): - if (s + MAX_BATCH_OPERATION_SIZE) < len(src_dest_pairs): - current_pairs = src_dest_pairs[s:s + MAX_BATCH_OPERATION_SIZE] - else: - current_pairs = src_dest_pairs[s:] - current_batch = self.client.batch(raise_exception=False) - with current_batch: - for pair in current_pairs: - src_bucket_name, src_blob_name = parse_gcs_path(pair[0]) - dest_bucket_name, dest_blob_name = parse_gcs_path(pair[1]) - src_bucket = self.client.get_bucket(src_bucket_name) - src_blob = src_bucket.get_blob(src_blob_name) - dest_bucket = self.client.get_bucket(dest_bucket_name) - - src_bucket.copy_blob(src_blob, dest_bucket, dest_blob_name) - - for i, pair in enumerate(current_pairs): - error_code = None - for j in range(4): - resp = current_batch._responses[4 * i + j] - if resp.status_code >= 400: - error_code = resp.status_code - break - final_results.append((pair[0], pair[1], error_code)) - - s += MAX_BATCH_OPERATION_SIZE - - return final_results + if not src_dest_pairs: + return [] + pair_to_request = {} + for pair in src_dest_pairs: + src_bucket, src_path = parse_gcs_path(pair[0]) + dest_bucket, dest_path = parse_gcs_path(pair[1]) + request = storage.StorageObjectsRewriteRequest( + sourceBucket=src_bucket, + sourceObject=src_path, + destinationBucket=dest_bucket, + destinationObject=dest_path, + destinationKmsKeyName=dest_kms_key_name, + maxBytesRewrittenPerCall=max_bytes_rewritten_per_call) + pair_to_request[pair] = request + pair_to_status = {} + while True: + pairs_in_batch = list(set(src_dest_pairs) - set(pair_to_status)) + if not pairs_in_batch: + break + batch_request = BatchApiRequest( + batch_url=GCS_BATCH_ENDPOINT, + retryable_codes=retry.SERVER_ERROR_OR_TIMEOUT_CODES, + response_encoding='utf-8') + for pair in pairs_in_batch: + batch_request.Add(self.client.objects, 'Rewrite', pair_to_request[pair]) + api_calls = batch_request.Execute(self.client._http) # pylint: disable=protected-access + for pair, api_call in zip(pairs_in_batch, api_calls): + src, dest = pair + response = api_call.response + if self._rewrite_cb is not None: + self._rewrite_cb(response) + if api_call.is_error: + exception = api_call.exception + # Translate 404 to the appropriate not found exception. + if isinstance(exception, HttpError) and exception.status_code == 404: + exception = ( + GcsIOError(errno.ENOENT, 'Source file not found: %s' % src)) + pair_to_status[pair] = exception + elif not response.done: + _LOGGER.debug( + 'Rewrite progress: %d of %d bytes, %s to %s', + response.totalBytesRewritten, + response.objectSize, + src, + dest) + pair_to_request[pair].rewriteToken = response.rewriteToken + else: + _LOGGER.debug('Rewrite done: %s to %s', src, dest) + pair_to_status[pair] = None + + return [(pair[0], pair[1], pair_to_status[pair]) for pair in src_dest_pairs] # We intentionally do not decorate this method with a retry, since the - # underlying copy and delete operations are already idempotent operations. + # underlying copy and delete operations are already idempotent operations + # protected by retry decorators. def copytree(self, src, dest): """Renames the given GCS "directory" recursively from src to dest. @@ -317,7 +465,8 @@ def copytree(self, src, dest): self.copy(entry, dest + rel_path) # We intentionally do not decorate this method with a retry, since the - # underlying copy and delete operations are already idempotent operations. + # underlying copy and delete operations are already idempotent operations + # protected by retry decorators. def rename(self, src, dest): """Renames the given GCS object from src to dest. @@ -335,10 +484,15 @@ def exists(self, path): path: GCS file path pattern in the form gs:///. """ try: - self._gcs_object(path) + self._gcs_object(path) # gcs object return True - except NotFound: - return False + except HttpError as http_error: + if http_error.status_code == 404: + # HTTP 404 indicates that the file did not exist + return False + else: + # We re-raise all other exceptions + raise def checksum(self, path): """Looks up the checksum of a GCS object. @@ -367,7 +521,7 @@ def kms_key(self, path): Returns: KMS key name of the GCS object as a string, or None if it doesn't have one. """ - return self._gcs_object(path).kms_key_name + return self._gcs_object(path).kmsKeyName def last_updated(self, path): """Returns the last updated epoch time of a single GCS object. @@ -394,10 +548,10 @@ def _status(self, path): file_status = {} if hasattr(gcs_object, 'crc32c'): file_status['checksum'] = gcs_object.crc32c - if hasattr(gcs_object, 'kms_key_name'): - file_status['kms_key'] = gcs_object.kms_key_name + if hasattr(gcs_object, 'kmsKeyName'): + file_status['kms_key'] = gcs_object.kmsKeyName if hasattr(gcs_object, 'updated'): - file_status['updated'] = self._updated_to_seconds(gcs_object.updated) + file_status['last_updated'] = self._updated_to_seconds(gcs_object.updated) if hasattr(gcs_object, 'size'): file_status['size'] = gcs_object.size return file_status @@ -412,13 +566,10 @@ def _gcs_object(self, path): Returns: GCS object. """ - bucket_name, blob_name = parse_gcs_path(path) - bucket = self.client.get_bucket(bucket_name) - blob = bucket.get_blob(blob_name) - if blob: - return blob - else: - raise NotFound('Object %s not found', path) + bucket, object_path = parse_gcs_path(path) + request = storage.StorageObjectsGetRequest( + bucket=bucket, object=object_path) + return self.client.objects.Get(request) @deprecated(since='2.45.0', current='list_files') def list_prefix(self, path, with_metadata=False): @@ -453,7 +604,8 @@ def list_files(self, path, with_metadata=False): ``with_metadata`` is True: generator of tuple(file name, tuple(size, timestamp)). """ - bucket_name, prefix = parse_gcs_path(path, object_optional=True) + bucket, prefix = parse_gcs_path(path, object_optional=True) + request = storage.StorageObjectsListRequest(bucket=bucket, prefix=prefix) file_info = set() counter = 0 start_time = time.time() @@ -461,26 +613,35 @@ def list_files(self, path, with_metadata=False): _LOGGER.debug("Starting the file information of the input") else: _LOGGER.debug("Starting the size estimation of the input") - bucket = self.client.get_bucket(bucket_name) - response = self.client.list_blobs(bucket, prefix=prefix) - for item in response: - file_name = 'gs://%s/%s' % (item.bucket.name, item.name) - if file_name not in file_info: - file_info.add(file_name) - counter += 1 - if counter % 10000 == 0: + while True: + response = retry.with_exponential_backoff( + retry_filter=retry.retry_on_server_errors_and_timeout_filter)( + self.client.objects.List)( + request) + + for item in response.items: + file_name = 'gs://%s/%s' % (item.bucket, item.name) + if file_name not in file_info: + file_info.add(file_name) + counter += 1 + if counter % 10000 == 0: + if with_metadata: + _LOGGER.info( + "Finished computing file information of: %s files", + len(file_info)) + else: + _LOGGER.info( + "Finished computing size of: %s files", len(file_info)) + if with_metadata: - _LOGGER.info( - "Finished computing file information of: %s files", - len(file_info)) + yield file_name, (item.size, self._updated_to_seconds(item.updated)) else: - _LOGGER.info("Finished computing size of: %s files", len(file_info)) - - if with_metadata: - yield file_name, (item.size, self._updated_to_seconds(item.updated)) - else: - yield file_name, item.size + yield file_name, item.size + if response.nextPageToken: + request.pageToken = response.nextPageToken + else: + break _LOGGER.log( # do not spam logs when list_prefix is likely used to check empty folder logging.INFO if counter > 0 else logging.DEBUG, @@ -496,18 +657,172 @@ def _updated_to_seconds(updated): updated.microsecond / 1000000.0) -class BeamBlobReader(BlobReader): - def __init__(self, blob, chunk_size=DEFAULT_READ_BUFFER_SIZE): - super().__init__(blob, chunk_size=chunk_size) - self.mode = "r" +class GcsDownloader(Downloader): + def __init__(self, client, path, buffer_size, get_project_number): + self._client = client + self._path = path + self._bucket, self._name = parse_gcs_path(path) + self._buffer_size = buffer_size + self._get_project_number = get_project_number + + # Create a request count metric + resource = resource_identifiers.GoogleCloudStorageBucket(self._bucket) + labels = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.get', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: self._bucket + } + project_number = self._get_project_number(self._bucket) + if project_number: + labels[monitoring_infos.GCS_PROJECT_ID_LABEL] = str(project_number) + else: + _LOGGER.debug( + 'Possibly missing storage.buckets.get permission to ' + 'bucket %s. Label %s is not added to the counter because it ' + 'cannot be identified.', + self._bucket, + monitoring_infos.GCS_PROJECT_ID_LABEL) + + service_call_metric = ServiceCallMetric( + request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, + base_labels=labels) + + # Get object state. + self._get_request = ( + storage.StorageObjectsGetRequest( + bucket=self._bucket, object=self._name)) + try: + metadata = self._get_object_metadata(self._get_request) + except HttpError as http_error: + service_call_metric.call(http_error) + if http_error.status_code == 404: + raise IOError(errno.ENOENT, 'Not found: %s' % self._path) + else: + _LOGGER.error( + 'HTTP error while requesting file %s: %s', self._path, http_error) + raise + else: + service_call_metric.call('ok') + + self._size = metadata.size + + # Ensure read is from file of the correct generation. + self._get_request.generation = metadata.generation + + # Initialize read buffer state. + self._download_stream = io.BytesIO() + self._downloader = transfer.Download( + self._download_stream, + auto_transfer=False, + chunksize=self._buffer_size, + num_retries=20) + try: + self._client.objects.Get(self._get_request, download=self._downloader) + service_call_metric.call('ok') + except HttpError as e: + service_call_metric.call(e) + raise -class BeamBlobWriter(BlobWriter): - def __init__( - self, blob, content_type, chunk_size=16 * 1024 * 1024, ignore_flush=True): - super().__init__( - blob, - content_type=content_type, - chunk_size=chunk_size, - ignore_flush=ignore_flush) - self.mode = "w" + @retry.with_exponential_backoff( + retry_filter=retry.retry_on_server_errors_and_timeout_filter) + def _get_object_metadata(self, get_request): + return self._client.objects.Get(get_request) + + @property + def size(self): + return self._size + + def get_range(self, start, end): + self._download_stream.seek(0) + self._download_stream.truncate(0) + self._downloader.GetRange(start, end - 1) + return self._download_stream.getvalue() + + +class GcsUploader(Uploader): + def __init__(self, client, path, mime_type, get_project_number): + self._client = client + self._path = path + self._bucket, self._name = parse_gcs_path(path) + self._mime_type = mime_type + self._get_project_number = get_project_number + + # Set up communication with child thread. + parent_conn, child_conn = multiprocessing.Pipe() + self._child_conn = child_conn + self._conn = parent_conn + + # Set up uploader. + self._insert_request = ( + storage.StorageObjectsInsertRequest( + bucket=self._bucket, name=self._name)) + self._upload = transfer.Upload( + PipeStream(self._child_conn), + self._mime_type, + chunksize=WRITE_CHUNK_SIZE) + self._upload.strategy = transfer.RESUMABLE_UPLOAD + + # Start uploading thread. + self._upload_thread = threading.Thread(target=self._start_upload) + self._upload_thread.daemon = True + self._upload_thread.last_error = None + self._upload_thread.start() + + # TODO(silviuc): Refactor so that retry logic can be applied. + # There is retry logic in the underlying transfer library but we should make + # it more explicit so we can control the retry parameters. + @retry.no_retries # Using no_retries marks this as an integration point. + def _start_upload(self): + # This starts the uploader thread. We are forced to run the uploader in + # another thread because the apitools uploader insists on taking a stream + # as input. Happily, this also means we get asynchronous I/O to GCS. + # + # The uploader by default transfers data in chunks of 1024 * 1024 bytes at + # a time, buffering writes until that size is reached. + + project_number = self._get_project_number(self._bucket) + + # Create a request count metric + resource = resource_identifiers.GoogleCloudStorageBucket(self._bucket) + labels = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.insert', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: self._bucket, + monitoring_infos.GCS_PROJECT_ID_LABEL: str(project_number) + } + service_call_metric = ServiceCallMetric( + request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, + base_labels=labels) + try: + self._client.objects.Insert(self._insert_request, upload=self._upload) + service_call_metric.call('ok') + except Exception as e: # pylint: disable=broad-except + service_call_metric.call(e) + _LOGGER.error( + 'Error in _start_upload while inserting file %s: %s', + self._path, + traceback.format_exc()) + self._upload_thread.last_error = e + finally: + self._child_conn.close() + + def put(self, data): + try: + self._conn.send_bytes(data.tobytes()) + except EOFError: + if self._upload_thread.last_error is not None: + raise self._upload_thread.last_error # pylint: disable=raising-bad-type + raise + + def finish(self): + self._conn.close() + # TODO(udim): Add timeout=DEFAULT_HTTP_TIMEOUT_SECONDS * 2 and raise if + # isAlive is True. + self._upload_thread.join() + # Check for exception since the last put() call. + if self._upload_thread.last_error is not None: + e = self._upload_thread.last_error + raise RuntimeError("Error while uploading file %s" % self._path) from e diff --git a/sdks/python/apache_beam/io/gcp/gcsio_integration_test.py b/sdks/python/apache_beam/io/gcp/gcsio_integration_test.py index a50145d84cae..4ffbea0ba026 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_integration_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_integration_test.py @@ -21,9 +21,20 @@ Instantiates a TestPipeline to get options such as GCP project name, but doesn't actually start a Beam pipeline or test any specific runner. -Run the following in 'sdks/python' directory to run these tests manually: - scripts/run_integration_test.sh \ - --test_opts apache_beam/io/gcp/gcsio_integration_test.py +Options: + --kms_key_name=projects//locations//keyRings/\ + /cryptoKeys//cryptoKeyVersions/ + Pass a Cloud KMS key name to test GCS operations using customer managed + encryption keys (CMEK). + +Cloud KMS permissions: +The project's Cloud Storage service account requires Encrypter/Decrypter +permissions for the key specified in --kms_key_name. + +To run these tests manually: + ./gradlew :sdks:python:test-suites:dataflow:integrationTest \ + -Dtests=apache_beam.io.gcp.gcsio_integration_test:GcsIOIntegrationTest \ + -DkmsKeyName=KMS_KEY_NAME """ # pytype: skip-file @@ -47,6 +58,9 @@ class GcsIOIntegrationTest(unittest.TestCase): INPUT_FILE = 'gs://dataflow-samples/shakespeare/kinglear.txt' + # Larger than 1MB to test maxBytesRewrittenPerCall. + # Also needs to be in a different region than the dest to take effect. + INPUT_FILE_LARGE = 'gs://apache-beam-samples-us-east1/wikipedia_edits/wiki_data-000000000000.json' # pylint: disable=line-too-long def setUp(self): self.test_pipeline = TestPipeline(is_integration_test=True) @@ -60,86 +74,126 @@ def setUp(self): self.gcs_tempdir = ( self.test_pipeline.get_option('temp_location') + '/gcs_it-' + str(uuid.uuid4())) + self.kms_key_name = self.test_pipeline.get_option('kms_key_name') self.gcsio = gcsio.GcsIO() def tearDown(self): FileSystems.delete([self.gcs_tempdir + '/']) - def _verify_copy(self, src, dest, dest_kms_key_name=None): - self.assertTrue( - FileSystems.exists(src), 'source file does not exist: %s' % src) - self.assertTrue( - FileSystems.exists(dest), - 'copied file not present in destination: %s' % dest) + def _verify_copy(self, src, dst, dst_kms_key_name=None): + self.assertTrue(FileSystems.exists(src), 'src does not exist: %s' % src) + self.assertTrue(FileSystems.exists(dst), 'dst does not exist: %s' % dst) src_checksum = self.gcsio.checksum(src) - dest_checksum = self.gcsio.checksum(dest) - self.assertEqual(src_checksum, dest_checksum) - actual_dest_kms_key = self.gcsio.kms_key(dest) - if actual_dest_kms_key is None: - self.assertEqual(actual_dest_kms_key, dest_kms_key_name) + dst_checksum = self.gcsio.checksum(dst) + self.assertEqual(src_checksum, dst_checksum) + actual_dst_kms_key = self.gcsio.kms_key(dst) + if actual_dst_kms_key is None: + self.assertEqual(actual_dst_kms_key, dst_kms_key_name) else: self.assertTrue( - actual_dest_kms_key.startswith(dest_kms_key_name), + actual_dst_kms_key.startswith(dst_kms_key_name), "got: %s, wanted startswith: %s" % - (actual_dest_kms_key, dest_kms_key_name)) + (actual_dst_kms_key, dst_kms_key_name)) + + def _test_copy( + self, + name, + kms_key_name=None, + max_bytes_rewritten_per_call=None, + src=None): + src = src or self.INPUT_FILE + dst = self.gcs_tempdir + '/%s' % name + extra_kwargs = {} + if max_bytes_rewritten_per_call is not None: + extra_kwargs['max_bytes_rewritten_per_call'] = ( + max_bytes_rewritten_per_call) + + self.gcsio.copy(src, dst, kms_key_name, **extra_kwargs) + self._verify_copy(src, dst, kms_key_name) @pytest.mark.it_postcommit def test_copy(self): - src = self.INPUT_FILE - dest = self.gcs_tempdir + '/test_copy' + self._test_copy("test_copy") - self.gcsio.copy(src, dest) - self._verify_copy(src, dest) + @pytest.mark.it_postcommit + def test_copy_kms(self): + if self.kms_key_name is None: + raise unittest.SkipTest('--kms_key_name not specified') + self._test_copy("test_copy_kms", self.kms_key_name) @pytest.mark.it_postcommit - def test_batch_copy_and_delete(self): + def test_copy_rewrite_token(self): + # Tests a multi-part copy (rewrite) operation. This is triggered by a + # combination of 3 conditions: + # - a large enough src + # - setting max_bytes_rewritten_per_call + # - setting kms_key_name + if self.kms_key_name is None: + raise unittest.SkipTest('--kms_key_name not specified') + + rewrite_responses = [] + self.gcsio._set_rewrite_response_callback( + lambda response: rewrite_responses.append(response)) + self._test_copy( + "test_copy_rewrite_token", + kms_key_name=self.kms_key_name, + max_bytes_rewritten_per_call=50 * 1024 * 1024, + src=self.INPUT_FILE_LARGE) + # Verify that there was a multi-part rewrite. + self.assertTrue(any(not r.done for r in rewrite_responses)) + + def _test_copy_batch( + self, + name, + kms_key_name=None, + max_bytes_rewritten_per_call=None, + src=None): num_copies = 10 - srcs = [self.INPUT_FILE] * num_copies - dests = [ - self.gcs_tempdir + '/test_copy_batch_%d' % i for i in range(num_copies) - ] - src_dest_pairs = list(zip(srcs, dests)) - - copy_results = self.gcsio.copy_batch(src_dest_pairs) - - self.assertEqual(len(copy_results), len(src_dest_pairs)) - - for pair, result in list(zip(src_dest_pairs, copy_results)): - self._verify_copy(pair[0], pair[1]) - self.assertEqual( - pair[0], - result[0], - 'copy source %s does not match %s' % (pair[0], str(result))) - self.assertEqual( - pair[1], - result[1], - 'copy destination %s does not match %s' % (pair[1], result[1])) - self.assertFalse( - result[2], - 'response code %s indicates that copy operation did not succeed' % - result[2]) - - delete_results = self.gcsio.delete_batch(dests) - - self.assertEqual(len(delete_results), len(dests)) - - for dest, result in list(zip(dests, delete_results)): - self.assertFalse( - FileSystems.exists(dest), 'deleted file still exists: %s' % dest) - self.assertEqual( - dest, - result[0], - 'delete path %s does not match %s' % (dest, result[0])) - self.assertFalse( - result[1], - 'response code %s indicates that delete operation did not succeed' % - result[1]) - - redelete_results = self.gcsio.delete_batch(dests) - - for dest, result in list(zip(dests, redelete_results)): - self.assertFalse( - result[1], 're-delete should not throw error: %s' % result[1]) + srcs = [src or self.INPUT_FILE] * num_copies + dsts = [self.gcs_tempdir + '/%s_%d' % (name, i) for i in range(num_copies)] + src_dst_pairs = list(zip(srcs, dsts)) + extra_kwargs = {} + if max_bytes_rewritten_per_call is not None: + extra_kwargs['max_bytes_rewritten_per_call'] = ( + max_bytes_rewritten_per_call) + + result_statuses = self.gcsio.copy_batch( + src_dst_pairs, kms_key_name, **extra_kwargs) + for status in result_statuses: + self.assertIsNone(status[2], status) + for _src, _dst in src_dst_pairs: + self._verify_copy(_src, _dst, kms_key_name) + + @pytest.mark.it_postcommit + def test_copy_batch(self): + self._test_copy_batch("test_copy_batch") + + @pytest.mark.it_postcommit + def test_copy_batch_kms(self): + if self.kms_key_name is None: + raise unittest.SkipTest('--kms_key_name not specified') + self._test_copy_batch("test_copy_batch_kms", self.kms_key_name) + + @pytest.mark.it_postcommit + def test_copy_batch_rewrite_token(self): + # Tests a multi-part copy (rewrite) operation. This is triggered by a + # combination of 3 conditions: + # - a large enough src + # - setting max_bytes_rewritten_per_call + # - setting kms_key_name + if self.kms_key_name is None: + raise unittest.SkipTest('--kms_key_name not specified') + + rewrite_responses = [] + self.gcsio._set_rewrite_response_callback( + lambda response: rewrite_responses.append(response)) + self._test_copy_batch( + "test_copy_batch_rewrite_token", + kms_key_name=self.kms_key_name, + max_bytes_rewritten_per_call=50 * 1024 * 1024, + src=self.INPUT_FILE_LARGE) + # Verify that there was a multi-part rewrite. + self.assertTrue(any(not r.done for r in rewrite_responses)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/io/gcp/gcsio_overrides.py b/sdks/python/apache_beam/io/gcp/gcsio_overrides.py new file mode 100644 index 000000000000..fc06cb28f1ad --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/gcsio_overrides.py @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# pytype: skip-file + +import logging +import math +import time + +from apache_beam.metrics.metric import Metrics +from apitools.base.py import exceptions +from apitools.base.py import http_wrapper +from apitools.base.py import util + +_LOGGER = logging.getLogger(__name__) + + +class GcsIOOverrides(object): + """Functions for overriding Google Cloud Storage I/O client.""" + + _THROTTLED_SECS = Metrics.counter('StorageV1', "cumulativeThrottlingSeconds") + + @classmethod + def retry_func(cls, retry_args): + # handling GCS download throttling errors (BEAM-7424) + if (isinstance(retry_args.exc, exceptions.BadStatusCodeError) and + retry_args.exc.status_code == http_wrapper.TOO_MANY_REQUESTS): + _LOGGER.debug( + 'Caught GCS quota error (%s), retrying.', retry_args.exc.status_code) + else: + return http_wrapper.HandleExceptionsAndRebuildHttpConnections(retry_args) + + http_wrapper.RebuildHttpConnections(retry_args.http) + _LOGGER.debug( + 'Retrying request to url %s after exception %s', + retry_args.http_request.url, + retry_args.exc) + sleep_seconds = util.CalculateWaitForRetry( + retry_args.num_retries, max_wait=retry_args.max_retry_wait) + cls._THROTTLED_SECS.inc(math.ceil(sleep_seconds)) + time.sleep(sleep_seconds) diff --git a/sdks/python/apache_beam/io/gcp/gcsio_test.py b/sdks/python/apache_beam/io/gcp/gcsio_test.py index f8b580c91c95..9cc5a9e1df0c 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_test.py @@ -18,147 +18,257 @@ """Tests for Google Cloud Storage client.""" # pytype: skip-file +import datetime +import errno +import io import logging import os +import random +import time import unittest -from datetime import datetime +from email.message import Message +import httplib2 import mock +# Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position +import apache_beam +from apache_beam.metrics import monitoring_infos +from apache_beam.metrics.execution import MetricsEnvironment +from apache_beam.metrics.metricbase import MetricName try: - from apache_beam.io.gcp import gcsio - from google.cloud.exceptions import BadRequest, NotFound + from apache_beam.io.gcp import gcsio, resource_identifiers + from apache_beam.io.gcp.internal.clients import storage + from apitools.base.py.exceptions import HttpError except ImportError: - NotFound = None + HttpError = None # pylint: enable=wrong-import-order, wrong-import-position DEFAULT_GCP_PROJECT = 'apache-beam-testing' +DEFAULT_PROJECT_NUMBER = 1 class FakeGcsClient(object): - # Fake storage client. + # Fake storage client. Usage in gcsio.py is client.objects.Get(...) and + # client.objects.Insert(...). def __init__(self): - self.buckets = {} + self.objects = FakeGcsObjects() + self.buckets = FakeGcsBuckets() + # Referenced in GcsIO.copy_batch() and GcsIO.delete_batch(). + self._http = object() - def create_bucket(self, name): - self.buckets[name] = FakeBucket(self, name) - return self.buckets[name] - def get_bucket(self, name): - if name in self.buckets: - return self.buckets[name] - else: - raise NotFound("Bucket not found") +class FakeFile(object): + def __init__( + self, bucket, obj, contents, generation, crc32c=None, last_updated=None): + self.bucket = bucket + self.object = obj + self.contents = contents + self.generation = generation + self.crc32c = crc32c + self.last_updated = last_updated + + def get_metadata(self): + last_updated_datetime = None + if self.last_updated: + last_updated_datetime = datetime.datetime.utcfromtimestamp( + self.last_updated) + + return storage.Object( + bucket=self.bucket, + name=self.object, + generation=self.generation, + size=len(self.contents), + crc32c=self.crc32c, + updated=last_updated_datetime) - def lookup_bucket(self, name): - if name in self.buckets: - return self.buckets[name] - else: - return self.create_bucket(name) - def batch(self): +class FakeGcsBuckets(object): + def __init__(self): pass - def add_file(self, bucket, blob, contents): - folder = self.lookup_bucket(bucket) - holder = folder.lookup_blob(blob) - holder.contents = contents + def get_bucket(self, bucket): + return storage.Bucket(name=bucket, projectNumber=DEFAULT_PROJECT_NUMBER) + + def Get(self, get_request): + return self.get_bucket(get_request.bucket) - def get_file(self, bucket, blob): - folder = self.get_bucket(bucket.name) - holder = folder.get_blob(blob.name) - return holder - def list_blobs(self, bucket_or_path, prefix=None): - bucket = self.get_bucket(bucket_or_path.name) - if not prefix: - return list(bucket.blobs.values()) +class FakeGcsObjects(object): + def __init__(self): + self.files = {} + # Store the last generation used for a given object name. Note that this + # has to persist even past the deletion of the object. + self.last_generation = {} + self.list_page_tokens = {} + self._fail_when_getting_metadata = [] + self._fail_when_reading = [] + + def add_file( + self, f, fail_when_getting_metadata=False, fail_when_reading=False): + self.files[(f.bucket, f.object)] = f + self.last_generation[(f.bucket, f.object)] = f.generation + if fail_when_getting_metadata: + self._fail_when_getting_metadata.append(f) + if fail_when_reading: + self._fail_when_reading.append(f) + + def get_file(self, bucket, obj): + return self.files.get((bucket, obj), None) + + def delete_file(self, bucket, obj): + del self.files[(bucket, obj)] + + def get_last_generation(self, bucket, obj): + return self.last_generation.get((bucket, obj), 0) + + def Get(self, get_request, download=None): # pylint: disable=invalid-name + f = self.get_file(get_request.bucket, get_request.object) + if f is None: + # Failing with an HTTP 404 if file does not exist. + raise HttpError({'status': 404}, None, None) + if download is None: + if f in self._fail_when_getting_metadata: + raise HttpError({'status': 429}, None, None) + return f.get_metadata() else: - output = [] - for name in list(bucket.blobs): - if name[0:len(prefix)] == prefix: - output.append(bucket.blobs[name]) - return output - - -class FakeBucket(object): - #Fake bucket for storing test blobs locally. - - def __init__(self, client, name): - self.client = client - self.name = name - self.blobs = {} - self.default_kms_key_name = None - self.client.buckets[name] = self - - def add_blob(self, blob): - self.blobs[blob.name] = blob - - def create_blob(self, name): - return FakeBlob(name, self) - - def copy_blob(self, blob, dest, new_name=None): - if not new_name: - new_name = blob.name - dest.blobs[new_name] = blob - dest.blobs[new_name].name = new_name - dest.blobs[new_name].bucket = dest - return dest.blobs[new_name] - - def get_blob(self, blob_name): - if blob_name in self.blobs: - return self.blobs[blob_name] + if f in self._fail_when_reading: + raise HttpError({'status': 429}, None, None) + stream = download.stream + + def get_range_callback(start, end): + if not 0 <= start <= end < len(f.contents): + raise ValueError( + 'start=%d end=%d len=%s' % (start, end, len(f.contents))) + stream.write(f.contents[start:end + 1]) + + download.GetRange = get_range_callback + + def Insert(self, insert_request, upload=None): # pylint: disable=invalid-name + assert upload is not None + generation = self.get_last_generation( + insert_request.bucket, insert_request.name) + 1 + f = FakeFile(insert_request.bucket, insert_request.name, b'', generation) + + # Stream data into file. + stream = upload.stream + data_list = [] + while True: + data = stream.read(1024 * 1024) + if not data: + break + data_list.append(data) + f.contents = b''.join(data_list) + + self.add_file(f) + + REWRITE_TOKEN = 'test_token' + + def Rewrite(self, rewrite_request): # pylint: disable=invalid-name + if rewrite_request.rewriteToken == self.REWRITE_TOKEN: + dest_object = storage.Object() + return storage.RewriteResponse( + done=True, + objectSize=100, + resource=dest_object, + totalBytesRewritten=100) + + src_file = self.get_file( + rewrite_request.sourceBucket, rewrite_request.sourceObject) + if not src_file: + raise HttpError( + httplib2.Response({'status': '404'}), + '404 Not Found', + 'https://fake/url') + generation = self.get_last_generation( + rewrite_request.destinationBucket, + rewrite_request.destinationObject) + 1 + dest_file = FakeFile( + rewrite_request.destinationBucket, + rewrite_request.destinationObject, + src_file.contents, + generation) + self.add_file(dest_file) + time.sleep(10) # time.sleep and time.time are mocked below. + return storage.RewriteResponse( + done=False, + objectSize=100, + rewriteToken=self.REWRITE_TOKEN, + totalBytesRewritten=5) + + def Delete(self, delete_request): # pylint: disable=invalid-name + # Here, we emulate the behavior of the GCS service in raising a 404 error + # if this object already exists. + if self.get_file(delete_request.bucket, delete_request.object): + self.delete_file(delete_request.bucket, delete_request.object) else: - return None - - def lookup_blob(self, name): - if name in self.blobs: - return self.blobs[name] + raise HttpError( + httplib2.Response({'status': '404'}), + '404 Not Found', + 'https://fake/url') + + def List(self, list_request): # pylint: disable=invalid-name + bucket = list_request.bucket + prefix = list_request.prefix or '' + matching_files = [] + for file_bucket, file_name in sorted(iter(self.files)): + if bucket == file_bucket and file_name.startswith(prefix): + file_object = self.files[(file_bucket, file_name)].get_metadata() + matching_files.append(file_object) + + # Handle pagination. + items_per_page = 5 + if not list_request.pageToken: + range_start = 0 else: - return self.create_blob(name) - - def set_default_kms_key_name(self, name): - self.default_kms_key_name = name - - def delete_blob(self, name): - if name in self.blobs: - del self.blobs[name] - - -class FakeBlob(object): - def __init__( - self, - name, - bucket, - size=0, - contents=None, - generation=1, - crc32c=None, - kms_key_name=None, - updated=None, - fail_when_getting_metadata=False, - fail_when_reading=False): - self.name = name - self.bucket = bucket - self.size = size - self.contents = contents - self._generation = generation - self.crc32c = crc32c - self.kms_key_name = kms_key_name - self.updated = updated - self._fail_when_getting_metadata = fail_when_getting_metadata - self._fail_when_reading = fail_when_reading - self.bucket.add_blob(self) - - def delete(self): - if self.name in self.bucket.blobs: - del self.bucket.blobs[self.name] - - -@unittest.skipIf(NotFound is None, 'GCP dependencies are not installed') + if list_request.pageToken not in self.list_page_tokens: + raise ValueError('Invalid page token.') + range_start = self.list_page_tokens[list_request.pageToken] + del self.list_page_tokens[list_request.pageToken] + + result = storage.Objects( + items=matching_files[range_start:range_start + items_per_page]) + if range_start + items_per_page < len(matching_files): + next_range_start = range_start + items_per_page + next_page_token = '_page_token_%s_%s_%d' % ( + bucket, prefix, next_range_start) + self.list_page_tokens[next_page_token] = next_range_start + result.nextPageToken = next_page_token + return result + + +class FakeApiCall(object): + def __init__(self, exception, response): + self.exception = exception + self.is_error = exception is not None + # Response for Rewrite: + self.response = response + + +class FakeBatchApiRequest(object): + def __init__(self, **unused_kwargs): + self.operations = [] + + def Add(self, service, method, request): # pylint: disable=invalid-name + self.operations.append((service, method, request)) + + def Execute(self, unused_http, **unused_kwargs): # pylint: disable=invalid-name + api_calls = [] + for service, method, request in self.operations: + exception = None + response = None + try: + response = getattr(service, method)(request) + except Exception as e: # pylint: disable=broad-except + exception = e + api_calls.append(FakeApiCall(exception, response)) + return api_calls + + +@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') class TestGCSPathParser(unittest.TestCase): BAD_GCS_PATHS = [ @@ -200,36 +310,34 @@ def __init__(self, project, region, kms_key=None): self.dataflow_kms_key = kms_key -@unittest.skipIf(NotFound is None, 'GCP dependencies are not installed') +@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') +@mock.patch.multiple( + 'time', time=mock.MagicMock(side_effect=range(100)), sleep=mock.MagicMock()) class TestGCSIO(unittest.TestCase): def _insert_random_file( self, client, path, - size=0, + size, + generation=1, crc32c=None, - kms_key_name=None, - updated=None, + last_updated=None, fail_when_getting_metadata=False, fail_when_reading=False): - bucket_name, blob_name = gcsio.parse_gcs_path(path) - bucket = client.lookup_bucket(bucket_name) - blob = FakeBlob( - blob_name, + bucket, name = gcsio.parse_gcs_path(path) + f = FakeFile( bucket, - size=size, - contents=os.urandom(size), + name, + os.urandom(size), + generation, crc32c=crc32c, - kms_key_name=kms_key_name, - updated=updated, - fail_when_getting_metadata=fail_when_getting_metadata, - fail_when_reading=fail_when_reading) - return blob + last_updated=last_updated) + client.objects.add_file(f, fail_when_getting_metadata, fail_when_reading) + return f def setUp(self): self.client = FakeGcsClient() self.gcs = gcsio.GcsIO(self.client) - self.client.create_bucket("gcsio-test") def test_default_bucket_name(self): self.assertEqual( @@ -243,6 +351,16 @@ def test_default_bucket_name_failure(self): DEFAULT_GCP_PROJECT, "us-central1", kms_key="kmskey!")), None) + def test_num_retries(self): + # BEAM-7424: update num_retries accordingly if storage_client is + # regenerated. + self.assertEqual(gcsio.GcsIO().client.num_retries, 20) + + def test_retry_func(self): + # BEAM-7667: update retry_func accordingly if storage_client is + # regenerated. + self.assertIsNotNone(gcsio.GcsIO().client.retry_func) + def test_exists(self): file_name = 'gs://gcsio-test/dummy_file' file_size = 1234 @@ -250,16 +368,17 @@ def test_exists(self): self.assertFalse(self.gcs.exists(file_name + 'xyz')) self.assertTrue(self.gcs.exists(file_name)) - @mock.patch.object(FakeBucket, 'get_blob') + @mock.patch.object(FakeGcsObjects, 'Get') def test_exists_failure(self, mock_get): # Raising an error other than 404. Raising 404 is a valid failure for # exists() call. - mock_get.side_effect = BadRequest("Try again") + mock_get.side_effect = HttpError({'status': 400}, None, None) file_name = 'gs://gcsio-test/dummy_file' file_size = 1234 self._insert_random_file(self.client, file_name, file_size) - with self.assertRaises(BadRequest): + with self.assertRaises(HttpError) as cm: self.gcs.exists(file_name) + self.assertEqual(400, cm.exception.status_code) def test_checksum(self): file_name = 'gs://gcsio-test/dummy_file' @@ -277,100 +396,179 @@ def test_size(self): self.assertTrue(self.gcs.exists(file_name)) self.assertEqual(1234, self.gcs.size(file_name)) - def test_kms_key(self): - file_name = 'gs://gcsio-test/dummy_file' - file_size = 1234 - kms_key_name = "dummy" - - self._insert_random_file( - self.client, file_name, file_size, kms_key_name=kms_key_name) - self.assertTrue(self.gcs.exists(file_name)) - self.assertEqual(kms_key_name, self.gcs.kms_key(file_name)) - def test_last_updated(self): file_name = 'gs://gcsio-test/dummy_file' file_size = 1234 - updated = datetime.fromtimestamp(123456.78) + last_updated = 123456.78 - self._insert_random_file(self.client, file_name, file_size, updated=updated) + self._insert_random_file( + self.client, file_name, file_size, last_updated=last_updated) self.assertTrue(self.gcs.exists(file_name)) - self.assertEqual( - gcsio.GcsIO._updated_to_seconds(updated), - self.gcs.last_updated(file_name)) + self.assertEqual(last_updated, self.gcs.last_updated(file_name)) def test_file_status(self): file_name = 'gs://gcsio-test/dummy_file' file_size = 1234 - updated = datetime.fromtimestamp(123456.78) + last_updated = 123456.78 checksum = 'deadbeef' self._insert_random_file( - self.client, file_name, file_size, updated=updated, crc32c=checksum) + self.client, + file_name, + file_size, + last_updated=last_updated, + crc32c=checksum) file_checksum = self.gcs.checksum(file_name) file_status = self.gcs._status(file_name) self.assertEqual(file_status['size'], file_size) self.assertEqual(file_status['checksum'], file_checksum) - self.assertEqual( - file_status['updated'], gcsio.GcsIO._updated_to_seconds(updated)) + self.assertEqual(file_status['last_updated'], last_updated) - def test_file_mode_calls(self): + def test_file_mode(self): file_name = 'gs://gcsio-test/dummy_mode_file' - self._insert_random_file(self.client, file_name) - with mock.patch('apache_beam.io.gcp.gcsio.BeamBlobWriter') as writer: - self.gcs.open(file_name, 'wb') - writer.assert_called() - with mock.patch('apache_beam.io.gcp.gcsio.BeamBlobReader') as reader: - self.gcs.open(file_name, 'rb') - reader.assert_called() + with self.gcs.open(file_name, 'wb') as f: + assert f.mode == 'wb' + with self.gcs.open(file_name, 'rb') as f: + assert f.mode == 'rb' def test_bad_file_modes(self): file_name = 'gs://gcsio-test/dummy_mode_file' - self._insert_random_file(self.client, file_name) with self.assertRaises(ValueError): self.gcs.open(file_name, 'w+') with self.assertRaises(ValueError): self.gcs.open(file_name, 'r+b') + def test_empty_batches(self): + self.assertEqual([], self.gcs.copy_batch([])) + self.assertEqual([], self.gcs.delete_batch([])) + def test_delete(self): file_name = 'gs://gcsio-test/delete_me' file_size = 1024 - bucket_name, blob_name = gcsio.parse_gcs_path(file_name) + # Test deletion of non-existent file. - bucket = self.client.get_bucket(bucket_name) self.gcs.delete(file_name) self._insert_random_file(self.client, file_name, file_size) - self.assertTrue(blob_name in bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(file_name) in self.client.objects.files) self.gcs.delete(file_name) - self.assertFalse(blob_name in bucket.blobs) + self.assertFalse( + gcsio.parse_gcs_path(file_name) in self.client.objects.files) + + @mock.patch( + 'apache_beam.io.gcp.gcsio.auth.get_service_credentials', + wraps=lambda pipeline_options: None) + @mock.patch('apache_beam.io.gcp.gcsio.get_new_http') + def test_user_agent_passed(self, get_new_http_mock, get_service_creds_mock): + client = gcsio.GcsIO() + try: + client.get_bucket('mabucket') + except: # pylint: disable=bare-except + # Ignore errors. The errors come from the fact that we did not mock + # the response from the API, so the overall get_bucket call fails + # soon after the GCS API is called. + pass + call = get_new_http_mock.return_value.request.mock_calls[-2] + self.assertIn( + "apache-beam/%s (GPN:Beam)" % apache_beam.__version__, + call[2]['headers']['User-Agent']) + + @mock.patch('apache_beam.io.gcp.gcsio.BatchApiRequest') + def test_delete_batch(self, *unused_args): + gcsio.BatchApiRequest = FakeBatchApiRequest + file_name_pattern = 'gs://gcsio-test/delete_me_%d' + file_size = 1024 + num_files = 10 + + # Test deletion of non-existent files. + result = self.gcs.delete_batch( + [file_name_pattern % i for i in range(num_files)]) + self.assertTrue(result) + for i, (file_name, exception) in enumerate(result): + self.assertEqual(file_name, file_name_pattern % i) + self.assertEqual(exception, None) + self.assertFalse(self.gcs.exists(file_name_pattern % i)) + + # Insert some files. + for i in range(num_files): + self._insert_random_file(self.client, file_name_pattern % i, file_size) + + # Check files inserted properly. + for i in range(num_files): + self.assertTrue(self.gcs.exists(file_name_pattern % i)) + + # Execute batch delete. + self.gcs.delete_batch([file_name_pattern % i for i in range(num_files)]) + + # Check files deleted properly. + for i in range(num_files): + self.assertFalse(self.gcs.exists(file_name_pattern % i)) def test_copy(self): src_file_name = 'gs://gcsio-test/source' dest_file_name = 'gs://gcsio-test/dest' - src_bucket_name, src_blob_name = gcsio.parse_gcs_path(src_file_name) - dest_bucket_name, dest_blob_name = gcsio.parse_gcs_path(dest_file_name) - src_bucket = self.client.lookup_bucket(src_bucket_name) - dest_bucket = self.client.lookup_bucket(dest_bucket_name) file_size = 1024 self._insert_random_file(self.client, src_file_name, file_size) - self.assertTrue(src_blob_name in src_bucket.blobs) - self.assertFalse(dest_blob_name in dest_bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertFalse( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) - self.gcs.copy(src_file_name, dest_file_name) + self.gcs.copy(src_file_name, dest_file_name, dest_kms_key_name='kms_key') - self.assertTrue(src_blob_name in src_bucket.blobs) - self.assertTrue(dest_blob_name in dest_bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertTrue( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) # Test copy of non-existent files. - with self.assertRaises(NotFound): + with self.assertRaisesRegex(HttpError, r'Not Found'): self.gcs.copy( 'gs://gcsio-test/non-existent', 'gs://gcsio-test/non-existent-destination') + @mock.patch('apache_beam.io.gcp.gcsio.BatchApiRequest') + def test_copy_batch(self, *unused_args): + gcsio.BatchApiRequest = FakeBatchApiRequest + from_name_pattern = 'gs://gcsio-test/copy_me_%d' + to_name_pattern = 'gs://gcsio-test/destination_%d' + file_size = 1024 + num_files = 10 + + result = self.gcs.copy_batch([(from_name_pattern % i, to_name_pattern % i) + for i in range(num_files)], + dest_kms_key_name='kms_key') + self.assertTrue(result) + for i, (src, dest, exception) in enumerate(result): + self.assertEqual(src, from_name_pattern % i) + self.assertEqual(dest, to_name_pattern % i) + self.assertTrue(isinstance(exception, IOError)) + self.assertEqual(exception.errno, errno.ENOENT) + self.assertFalse(self.gcs.exists(from_name_pattern % i)) + self.assertFalse(self.gcs.exists(to_name_pattern % i)) + + # Insert some files. + for i in range(num_files): + self._insert_random_file(self.client, from_name_pattern % i, file_size) + + # Check files inserted properly. + for i in range(num_files): + self.assertTrue(self.gcs.exists(from_name_pattern % i)) + + # Execute batch copy. + self.gcs.copy_batch([(from_name_pattern % i, to_name_pattern % i) + for i in range(num_files)]) + + # Check files copied properly. + for i in range(num_files): + self.assertTrue(self.gcs.exists(from_name_pattern % i)) + self.assertTrue(self.gcs.exists(to_name_pattern % i)) + def test_copytree(self): src_dir_name = 'gs://gcsio-test/source/' dest_dir_name = 'gs://gcsio-test/dest/' @@ -379,62 +577,204 @@ def test_copytree(self): for path in paths: src_file_name = src_dir_name + path dest_file_name = dest_dir_name + path - src_bucket_name, src_blob_name = gcsio.parse_gcs_path(src_file_name) - dest_bucket_name, dest_blob_name = gcsio.parse_gcs_path(dest_file_name) - src_bucket = self.client.lookup_bucket(src_bucket_name) - dest_bucket = self.client.lookup_bucket(dest_bucket_name) - file_size = 1024 self._insert_random_file(self.client, src_file_name, file_size) - self.assertTrue(src_blob_name in src_bucket.blobs) - self.assertFalse(dest_blob_name in dest_bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertFalse( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) self.gcs.copytree(src_dir_name, dest_dir_name) for path in paths: src_file_name = src_dir_name + path dest_file_name = dest_dir_name + path - src_bucket_name, src_blob_name = gcsio.parse_gcs_path(src_file_name) - dest_bucket_name, dest_blob_name = gcsio.parse_gcs_path(dest_file_name) - src_bucket = self.client.lookup_bucket(src_bucket_name) - dest_bucket = self.client.lookup_bucket(dest_bucket_name) - self.assertTrue(src_blob_name in src_bucket.blobs) - self.assertTrue(dest_blob_name in dest_bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertTrue( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) def test_rename(self): src_file_name = 'gs://gcsio-test/source' dest_file_name = 'gs://gcsio-test/dest' - src_bucket_name, src_blob_name = gcsio.parse_gcs_path(src_file_name) - dest_bucket_name, dest_blob_name = gcsio.parse_gcs_path(dest_file_name) file_size = 1024 self._insert_random_file(self.client, src_file_name, file_size) - src_bucket = self.client.lookup_bucket(src_bucket_name) - dest_bucket = self.client.lookup_bucket(dest_bucket_name) - self.assertTrue(src_blob_name in src_bucket.blobs) - self.assertFalse(dest_blob_name in dest_bucket.blobs) + self.assertTrue( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertFalse( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) self.gcs.rename(src_file_name, dest_file_name) - self.assertFalse(src_blob_name in src_bucket.blobs) - self.assertTrue(dest_blob_name in dest_bucket.blobs) + self.assertFalse( + gcsio.parse_gcs_path(src_file_name) in self.client.objects.files) + self.assertTrue( + gcsio.parse_gcs_path(dest_file_name) in self.client.objects.files) + + def test_full_file_read(self): + file_name = 'gs://gcsio-test/full_file' + file_size = 5 * 1024 * 1024 + 100 + random_file = self._insert_random_file(self.client, file_name, file_size) + f = self.gcs.open(file_name) + self.assertEqual(f.mode, 'r') + f.seek(0, os.SEEK_END) + self.assertEqual(f.tell(), file_size) + self.assertEqual(f.read(), b'') + f.seek(0) + self.assertEqual(f.read(), random_file.contents) + + def test_file_random_seek(self): + file_name = 'gs://gcsio-test/seek_file' + file_size = 5 * 1024 * 1024 - 100 + random_file = self._insert_random_file(self.client, file_name, file_size) + + f = self.gcs.open(file_name) + random.seed(0) + for _ in range(0, 10): + a = random.randint(0, file_size - 1) + b = random.randint(0, file_size - 1) + start, end = min(a, b), max(a, b) + f.seek(start) + self.assertEqual(f.tell(), start) + self.assertEqual( + f.read(end - start + 1), random_file.contents[start:end + 1]) + self.assertEqual(f.tell(), end + 1) + + def test_file_iterator(self): + file_name = 'gs://gcsio-test/iterating_file' + lines = [] + line_count = 10 + for _ in range(line_count): + line_length = random.randint(100, 500) + line = os.urandom(line_length).replace(b'\n', b' ') + b'\n' + lines.append(line) + + contents = b''.join(lines) + bucket, name = gcsio.parse_gcs_path(file_name) + self.client.objects.add_file(FakeFile(bucket, name, contents, 1)) + + f = self.gcs.open(file_name) + + read_lines = 0 + for line in f: + read_lines += 1 - def test_file_buffered_read_call(self): + self.assertEqual(read_lines, line_count) + + def test_file_read_line(self): file_name = 'gs://gcsio-test/read_line_file' + lines = [] + + # Set a small buffer size to exercise refilling the buffer. + # First line is carefully crafted so the newline falls as the last character + # of the buffer to exercise this code path. read_buffer_size = 1024 - self._insert_random_file(self.client, file_name, 10240) + lines.append(b'x' * 1023 + b'\n') + + for _ in range(1, 1000): + line_length = random.randint(100, 500) + line = os.urandom(line_length).replace(b'\n', b' ') + b'\n' + lines.append(line) + contents = b''.join(lines) + + file_size = len(contents) + bucket, name = gcsio.parse_gcs_path(file_name) + self.client.objects.add_file(FakeFile(bucket, name, contents, 1)) + + f = self.gcs.open(file_name, read_buffer_size=read_buffer_size) + + # Test read of first two lines. + f.seek(0) + self.assertEqual(f.readline(), lines[0]) + self.assertEqual(f.tell(), len(lines[0])) + self.assertEqual(f.readline(), lines[1]) + + # Test read at line boundary. + f.seek(file_size - len(lines[-1]) - 1) + self.assertEqual(f.readline(), b'\n') + + # Test read at end of file. + f.seek(file_size) + self.assertEqual(f.readline(), b'') + + # Test reads at random positions. + random.seed(0) + for _ in range(0, 10): + start = random.randint(0, file_size - 1) + line_index = 0 + # Find line corresponding to start index. + chars_left = start + while True: + next_line_length = len(lines[line_index]) + if chars_left - next_line_length < 0: + break + chars_left -= next_line_length + line_index += 1 + f.seek(start) + self.assertEqual(f.readline(), lines[line_index][chars_left:]) + + def test_file_write(self): + file_name = 'gs://gcsio-test/write_file' + file_size = 5 * 1024 * 1024 + 2000 + contents = os.urandom(file_size) + f = self.gcs.open(file_name, 'w') + self.assertEqual(f.mode, 'w') + f.write(contents[0:1000]) + f.write(contents[1000:1024 * 1024]) + f.write(contents[1024 * 1024:]) + f.close() + bucket, name = gcsio.parse_gcs_path(file_name) + self.assertEqual( + self.client.objects.get_file(bucket, name).contents, contents) + + def test_file_close(self): + file_name = 'gs://gcsio-test/close_file' + file_size = 5 * 1024 * 1024 + 2000 + contents = os.urandom(file_size) + f = self.gcs.open(file_name, 'w') + self.assertEqual(f.mode, 'w') + f.write(contents) + f.close() + f.close() # This should not crash. + bucket, name = gcsio.parse_gcs_path(file_name) + self.assertEqual( + self.client.objects.get_file(bucket, name).contents, contents) + + def test_file_flush(self): + file_name = 'gs://gcsio-test/flush_file' + file_size = 5 * 1024 * 1024 + 2000 + contents = os.urandom(file_size) + bucket, name = gcsio.parse_gcs_path(file_name) + f = self.gcs.open(file_name, 'w') + self.assertEqual(f.mode, 'w') + f.write(contents[0:1000]) + f.flush() + f.write(contents[1000:1024 * 1024]) + f.flush() + f.flush() # Should be a NOOP. + f.write(contents[1024 * 1024:]) + f.close() # This should already call the equivalent of flush() in its body. + self.assertEqual( + self.client.objects.get_file(bucket, name).contents, contents) - bucket_name, blob_name = gcsio.parse_gcs_path(file_name) - bucket = self.client.get_bucket(bucket_name) - blob = bucket.get_blob(blob_name) + def test_context_manager(self): + # Test writing with a context manager. + file_name = 'gs://gcsio-test/context_manager_file' + file_size = 1024 + contents = os.urandom(file_size) + with self.gcs.open(file_name, 'w') as f: + f.write(contents) + bucket, name = gcsio.parse_gcs_path(file_name) + self.assertEqual( + self.client.objects.get_file(bucket, name).contents, contents) - with mock.patch('apache_beam.io.gcp.gcsio.BeamBlobReader') as reader: - self.gcs.open(file_name, read_buffer_size=read_buffer_size) - reader.assert_called_with(blob, chunk_size=read_buffer_size) + # Test reading with a context manager. + with self.gcs.open(file_name) as f: + self.assertEqual(f.read(), contents) - def test_file_write_call(self): - file_name = 'gs://gcsio-test/write_file' - with mock.patch('apache_beam.io.gcp.gcsio.BeamBlobWriter') as writer: - self.gcs.open(file_name, 'w') - writer.assert_called() + # Test that exceptions are not swallowed by the context manager. + with self.assertRaises(ZeroDivisionError): + with self.gcs.open(file_name) as f: + f.read(0 // 0) def test_list_prefix(self): bucket_name = 'gcsio-test' @@ -472,11 +812,141 @@ def test_list_prefix(self): set(self.gcs.list_prefix(file_pattern).items()), set(expected_file_names)) + def test_mime_binary_encoding(self): + # This test verifies that the MIME email_generator library works properly + # and does not corrupt '\r\n' during uploads (the patch to apitools in + # Python 3 is applied in io/gcp/__init__.py). + from apitools.base.py.transfer import email_generator + generator_cls = email_generator.BytesGenerator + output_buffer = io.BytesIO() + generator = generator_cls(output_buffer) + test_msg = 'a\nb\r\nc\n\r\n\n\nd' + message = Message() + message.set_payload(test_msg) + generator._handle_text(message) + self.assertEqual(test_msg.encode('ascii'), output_buffer.getvalue()) + + def test_downloader_monitoring_info(self): + # Clear the process wide metric container. + MetricsEnvironment.process_wide_container().reset() + + file_name = 'gs://gcsio-metrics-test/dummy_mode_file' + file_size = 5 * 1024 * 1024 + 100 + random_file = self._insert_random_file(self.client, file_name, file_size) + self.gcs.open(file_name, 'r') + + resource = resource_identifiers.GoogleCloudStorageBucket(random_file.bucket) + labels = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.get', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: random_file.bucket, + monitoring_infos.GCS_PROJECT_ID_LABEL: str(DEFAULT_PROJECT_NUMBER), + monitoring_infos.STATUS_LABEL: 'ok' + } + + metric_name = MetricName( + None, None, urn=monitoring_infos.API_REQUEST_COUNT_URN, labels=labels) + metric_value = MetricsEnvironment.process_wide_container().get_counter( + metric_name).get_cumulative() + + self.assertEqual(metric_value, 2) + + @mock.patch.object(FakeGcsBuckets, 'Get') + def test_downloader_fail_to_get_project_number(self, mock_get): + # Raising an error when listing GCS Bucket so that project number fails to + # be retrieved. + mock_get.side_effect = HttpError({'status': 403}, None, None) + # Clear the process wide metric container. + MetricsEnvironment.process_wide_container().reset() + + file_name = 'gs://gcsio-metrics-test/dummy_mode_file' + file_size = 5 * 1024 * 1024 + 100 + random_file = self._insert_random_file(self.client, file_name, file_size) + self.gcs.open(file_name, 'r') + + resource = resource_identifiers.GoogleCloudStorageBucket(random_file.bucket) + labels = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.get', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: random_file.bucket, + monitoring_infos.GCS_PROJECT_ID_LABEL: str(DEFAULT_PROJECT_NUMBER), + monitoring_infos.STATUS_LABEL: 'ok' + } + + metric_name = MetricName( + None, None, urn=monitoring_infos.API_REQUEST_COUNT_URN, labels=labels) + metric_value = MetricsEnvironment.process_wide_container().get_counter( + metric_name).get_cumulative() + + self.assertEqual(metric_value, 0) + + labels_without_project_id = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.get', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: random_file.bucket, + monitoring_infos.STATUS_LABEL: 'ok' + } + metric_name = MetricName( + None, + None, + urn=monitoring_infos.API_REQUEST_COUNT_URN, + labels=labels_without_project_id) + metric_value = MetricsEnvironment.process_wide_container().get_counter( + metric_name).get_cumulative() + + self.assertEqual(metric_value, 2) + def test_downloader_fail_non_existent_object(self): file_name = 'gs://gcsio-metrics-test/dummy_mode_file' - with self.assertRaises(NotFound): + with self.assertRaises(IOError): + self.gcs.open(file_name, 'r') + + def test_downloader_fail_when_getting_metadata(self): + file_name = 'gs://gcsio-metrics-test/dummy_mode_file' + file_size = 5 * 1024 * 1024 + 100 + self._insert_random_file( + self.client, file_name, file_size, fail_when_getting_metadata=True) + with self.assertRaises(HttpError): + self.gcs.open(file_name, 'r') + + def test_downloader_fail_when_reading(self): + file_name = 'gs://gcsio-metrics-test/dummy_mode_file' + file_size = 5 * 1024 * 1024 + 100 + self._insert_random_file( + self.client, file_name, file_size, fail_when_reading=True) + with self.assertRaises(HttpError): self.gcs.open(file_name, 'r') + def test_uploader_monitoring_info(self): + # Clear the process wide metric container. + MetricsEnvironment.process_wide_container().reset() + + file_name = 'gs://gcsio-metrics-test/dummy_mode_file' + file_size = 5 * 1024 * 1024 + 100 + random_file = self._insert_random_file(self.client, file_name, file_size) + f = self.gcs.open(file_name, 'w') + + resource = resource_identifiers.GoogleCloudStorageBucket(random_file.bucket) + labels = { + monitoring_infos.SERVICE_LABEL: 'Storage', + monitoring_infos.METHOD_LABEL: 'Objects.insert', + monitoring_infos.RESOURCE_LABEL: resource, + monitoring_infos.GCS_BUCKET_LABEL: random_file.bucket, + monitoring_infos.GCS_PROJECT_ID_LABEL: str(DEFAULT_PROJECT_NUMBER), + monitoring_infos.STATUS_LABEL: 'ok' + } + + f.close() + metric_name = MetricName( + None, None, urn=monitoring_infos.API_REQUEST_COUNT_URN, labels=labels) + metric_value = MetricsEnvironment.process_wide_container().get_counter( + metric_name).get_cumulative() + + self.assertEqual(metric_value, 1) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/storage/__init__.py b/sdks/python/apache_beam/io/gcp/internal/clients/storage/__init__.py new file mode 100644 index 000000000000..b37a4b57c115 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/internal/clients/storage/__init__.py @@ -0,0 +1,33 @@ +# +# 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. +# + +"""Common imports for generated storage client library.""" +# pylint:disable=wildcard-import + +import pkgutil + +# Protect against environments where apitools library is not available. +# pylint: disable=wrong-import-order, wrong-import-position +try: + from apitools.base.py import * + from apache_beam.io.gcp.internal.clients.storage.storage_v1_client import * + from apache_beam.io.gcp.internal.clients.storage.storage_v1_messages import * +except ImportError: + pass +# pylint: enable=wrong-import-order, wrong-import-position + +__path__ = pkgutil.extend_path(__path__, __name__) # type: ignore diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py new file mode 100644 index 000000000000..e5b7c0268ec8 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py @@ -0,0 +1,1517 @@ +# +# 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. +# + +"""Generated client library for storage version v1.""" +# NOTE: This file is autogenerated and should not be edited by hand. + +from apitools.base.py import base_api + +from apache_beam.io.gcp.gcsio_overrides import GcsIOOverrides +from apache_beam.io.gcp.internal.clients.storage import \ + storage_v1_messages as messages + + +class StorageV1(base_api.BaseApiClient): + """Generated client library for service storage version v1.""" + + MESSAGES_MODULE = messages + BASE_URL = u'https://www.googleapis.com/storage/v1/' + + _PACKAGE = u'storage' + _SCOPES = [ + u'https://www.googleapis.com/auth/cloud-platform', + u'https://www.googleapis.com/auth/cloud-platform.read-only', + u'https://www.googleapis.com/auth/devstorage.full_control', + u'https://www.googleapis.com/auth/devstorage.read_only', + u'https://www.googleapis.com/auth/devstorage.read_write' + ] + _VERSION = u'v1' + _CLIENT_ID = '1042881264118.apps.googleusercontent.com' + _CLIENT_SECRET = 'x_Tw5K8nnjoRAqULM9PFAC2b' + _USER_AGENT = 'x_Tw5K8nnjoRAqULM9PFAC2b' + _CLIENT_CLASS_NAME = u'StorageV1' + _URL_VERSION = u'v1' + _API_KEY = None + + def __init__( + self, + url='', + credentials=None, + get_credentials=True, + http=None, + model=None, + log_request=False, + log_response=False, + credentials_args=None, + default_global_params=None, + additional_http_headers=None, + response_encoding=None): + """Create a new storage handle.""" + url = url or self.BASE_URL + super().__init__( + url, + credentials=credentials, + get_credentials=get_credentials, + http=http, + model=model, + log_request=log_request, + log_response=log_response, + num_retries=20, + credentials_args=credentials_args, + default_global_params=default_global_params, + additional_http_headers=additional_http_headers, + retry_func=GcsIOOverrides.retry_func, + response_encoding=response_encoding) + self.bucketAccessControls = self.BucketAccessControlsService(self) + self.buckets = self.BucketsService(self) + self.channels = self.ChannelsService(self) + self.defaultObjectAccessControls = self.DefaultObjectAccessControlsService( + self) + self.notifications = self.NotificationsService(self) + self.objectAccessControls = self.ObjectAccessControlsService(self) + self.objects = self.ObjectsService(self) + self.projects_serviceAccount = self.ProjectsServiceAccountService(self) + self.projects = self.ProjectsService(self) + + class BucketAccessControlsService(base_api.BaseApiService): + """Service class for the bucketAccessControls resource.""" + + _NAME = u'bucketAccessControls' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Delete(self, request, global_params=None): + r"""Permanently deletes the ACL entry for the specified entity on the specified bucket. + + Args: + request: (StorageBucketAccessControlsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageBucketAccessControlsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.bucketAccessControls.delete', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl/{entity}', + request_field='', + request_type_name=u'StorageBucketAccessControlsDeleteRequest', + response_type_name=u'StorageBucketAccessControlsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns the ACL entry for the specified entity on the specified bucket. + + Args: + request: (StorageBucketAccessControlsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BucketAccessControl) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.bucketAccessControls.get', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl/{entity}', + request_field='', + request_type_name=u'StorageBucketAccessControlsGetRequest', + response_type_name=u'BucketAccessControl', + supports_download=False, + ) + + def Insert(self, request, global_params=None): + r"""Creates a new ACL entry on the specified bucket. + + Args: + request: (StorageBucketAccessControlsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BucketAccessControl) The response message. + """ + config = self.GetMethodConfig('Insert') + return self._RunMethod(config, request, global_params=global_params) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.bucketAccessControls.insert', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl', + request_field=u'bucketAccessControl', + request_type_name=u'StorageBucketAccessControlsInsertRequest', + response_type_name=u'BucketAccessControl', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves ACL entries on the specified bucket. + + Args: + request: (StorageBucketAccessControlsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BucketAccessControls) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.bucketAccessControls.list', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl', + request_field='', + request_type_name=u'StorageBucketAccessControlsListRequest', + response_type_name=u'BucketAccessControls', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Patches an ACL entry on the specified bucket. + + Args: + request: (StorageBucketAccessControlsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BucketAccessControl) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PATCH', + method_id=u'storage.bucketAccessControls.patch', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl/{entity}', + request_field=u'bucketAccessControl', + request_type_name=u'StorageBucketAccessControlsPatchRequest', + response_type_name=u'BucketAccessControl', + supports_download=False, + ) + + def Update(self, request, global_params=None): + r"""Updates an ACL entry on the specified bucket. + + Args: + request: (StorageBucketAccessControlsUpdateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BucketAccessControl) The response message. + """ + config = self.GetMethodConfig('Update') + return self._RunMethod(config, request, global_params=global_params) + + Update.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.bucketAccessControls.update', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/acl/{entity}', + request_field=u'bucketAccessControl', + request_type_name=u'StorageBucketAccessControlsUpdateRequest', + response_type_name=u'BucketAccessControl', + supports_download=False, + ) + + class BucketsService(base_api.BaseApiService): + """Service class for the buckets resource.""" + + _NAME = u'buckets' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Delete(self, request, global_params=None): + r"""Permanently deletes an empty bucket. + + Args: + request: (StorageBucketsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageBucketsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.buckets.delete', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params= + [u'ifMetagenerationMatch', u'ifMetagenerationNotMatch', u'userProject'], + relative_path=u'b/{bucket}', + request_field='', + request_type_name=u'StorageBucketsDeleteRequest', + response_type_name=u'StorageBucketsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns metadata for the specified bucket. + + Args: + request: (StorageBucketsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Bucket) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.buckets.get', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}', + request_field='', + request_type_name=u'StorageBucketsGetRequest', + response_type_name=u'Bucket', + supports_download=False, + ) + + def GetIamPolicy(self, request, global_params=None): + r"""Returns an IAM policy for the specified bucket. + + Args: + request: (StorageBucketsGetIamPolicyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Policy) The response message. + """ + config = self.GetMethodConfig('GetIamPolicy') + return self._RunMethod(config, request, global_params=global_params) + + GetIamPolicy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.buckets.getIamPolicy', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/iam', + request_field='', + request_type_name=u'StorageBucketsGetIamPolicyRequest', + response_type_name=u'Policy', + supports_download=False, + ) + + def Insert(self, request, global_params=None): + r"""Creates a new bucket. + + Args: + request: (StorageBucketsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Bucket) The response message. + """ + config = self.GetMethodConfig('Insert') + return self._RunMethod(config, request, global_params=global_params) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.buckets.insert', + ordered_params=[u'project'], + path_params=[], + query_params=[ + u'predefinedAcl', + u'predefinedDefaultObjectAcl', + u'project', + u'projection', + u'userProject' + ], + relative_path=u'b', + request_field=u'bucket', + request_type_name=u'StorageBucketsInsertRequest', + response_type_name=u'Bucket', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves a list of buckets for a given project. + + Args: + request: (StorageBucketsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Buckets) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.buckets.list', + ordered_params=[u'project'], + path_params=[], + query_params=[ + u'maxResults', + u'pageToken', + u'prefix', + u'project', + u'projection', + u'userProject' + ], + relative_path=u'b', + request_field='', + request_type_name=u'StorageBucketsListRequest', + response_type_name=u'Buckets', + supports_download=False, + ) + + def LockRetentionPolicy(self, request, global_params=None): + r"""Locks retention policy on a bucket. + + Args: + request: (StorageBucketsLockRetentionPolicyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Bucket) The response message. + """ + config = self.GetMethodConfig('LockRetentionPolicy') + return self._RunMethod(config, request, global_params=global_params) + + LockRetentionPolicy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.buckets.lockRetentionPolicy', + ordered_params=[u'bucket', u'ifMetagenerationMatch'], + path_params=[u'bucket'], + query_params=[u'ifMetagenerationMatch', u'userProject'], + relative_path=u'b/{bucket}/lockRetentionPolicy', + request_field='', + request_type_name=u'StorageBucketsLockRetentionPolicyRequest', + response_type_name=u'Bucket', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Patches a bucket. Changes to the bucket will be readable immediately after writing, but configuration changes may take time to propagate. + + Args: + request: (StorageBucketsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Bucket) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PATCH', + method_id=u'storage.buckets.patch', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'predefinedAcl', + u'predefinedDefaultObjectAcl', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}', + request_field=u'bucketResource', + request_type_name=u'StorageBucketsPatchRequest', + response_type_name=u'Bucket', + supports_download=False, + ) + + def SetIamPolicy(self, request, global_params=None): + r"""Updates an IAM policy for the specified bucket. + + Args: + request: (StorageBucketsSetIamPolicyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Policy) The response message. + """ + config = self.GetMethodConfig('SetIamPolicy') + return self._RunMethod(config, request, global_params=global_params) + + SetIamPolicy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.buckets.setIamPolicy', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/iam', + request_field=u'policy', + request_type_name=u'StorageBucketsSetIamPolicyRequest', + response_type_name=u'Policy', + supports_download=False, + ) + + def TestIamPermissions(self, request, global_params=None): + r"""Tests a set of permissions on the given bucket to see which, if any, are held by the caller. + + Args: + request: (StorageBucketsTestIamPermissionsRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (TestIamPermissionsResponse) The response message. + """ + config = self.GetMethodConfig('TestIamPermissions') + return self._RunMethod(config, request, global_params=global_params) + + TestIamPermissions.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.buckets.testIamPermissions', + ordered_params=[u'bucket', u'permissions'], + path_params=[u'bucket'], + query_params=[u'permissions', u'userProject'], + relative_path=u'b/{bucket}/iam/testPermissions', + request_field='', + request_type_name=u'StorageBucketsTestIamPermissionsRequest', + response_type_name=u'TestIamPermissionsResponse', + supports_download=False, + ) + + def Update(self, request, global_params=None): + r"""Updates a bucket. Changes to the bucket will be readable immediately after writing, but configuration changes may take time to propagate. + + Args: + request: (StorageBucketsUpdateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Bucket) The response message. + """ + config = self.GetMethodConfig('Update') + return self._RunMethod(config, request, global_params=global_params) + + Update.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.buckets.update', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'predefinedAcl', + u'predefinedDefaultObjectAcl', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}', + request_field=u'bucketResource', + request_type_name=u'StorageBucketsUpdateRequest', + response_type_name=u'Bucket', + supports_download=False, + ) + + class ChannelsService(base_api.BaseApiService): + """Service class for the channels resource.""" + + _NAME = u'channels' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Stop(self, request, global_params=None): + r"""Stop watching resources through this channel. + + Args: + request: (Channel) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageChannelsStopResponse) The response message. + """ + config = self.GetMethodConfig('Stop') + return self._RunMethod(config, request, global_params=global_params) + + Stop.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.channels.stop', + ordered_params=[], + path_params=[], + query_params=[], + relative_path=u'channels/stop', + request_field='', + request_type_name=u'Channel', + response_type_name=u'StorageChannelsStopResponse', + supports_download=False, + ) + + class DefaultObjectAccessControlsService(base_api.BaseApiService): + """Service class for the defaultObjectAccessControls resource.""" + + _NAME = u'defaultObjectAccessControls' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Delete(self, request, global_params=None): + r"""Permanently deletes the default object ACL entry for the specified entity on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageDefaultObjectAccessControlsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.defaultObjectAccessControls.delete', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl/{entity}', + request_field='', + request_type_name=u'StorageDefaultObjectAccessControlsDeleteRequest', + response_type_name=u'StorageDefaultObjectAccessControlsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns the default object ACL entry for the specified entity on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.defaultObjectAccessControls.get', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl/{entity}', + request_field='', + request_type_name=u'StorageDefaultObjectAccessControlsGetRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def Insert(self, request, global_params=None): + r"""Creates a new default object ACL entry on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Insert') + return self._RunMethod(config, request, global_params=global_params) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.defaultObjectAccessControls.insert', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl', + request_field=u'objectAccessControl', + request_type_name=u'StorageDefaultObjectAccessControlsInsertRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves default object ACL entries on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControls) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.defaultObjectAccessControls.list', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params= + [u'ifMetagenerationMatch', u'ifMetagenerationNotMatch', u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl', + request_field='', + request_type_name=u'StorageDefaultObjectAccessControlsListRequest', + response_type_name=u'ObjectAccessControls', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Patches a default object ACL entry on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PATCH', + method_id=u'storage.defaultObjectAccessControls.patch', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl/{entity}', + request_field=u'objectAccessControl', + request_type_name=u'StorageDefaultObjectAccessControlsPatchRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def Update(self, request, global_params=None): + r"""Updates a default object ACL entry on the specified bucket. + + Args: + request: (StorageDefaultObjectAccessControlsUpdateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Update') + return self._RunMethod(config, request, global_params=global_params) + + Update.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.defaultObjectAccessControls.update', + ordered_params=[u'bucket', u'entity'], + path_params=[u'bucket', u'entity'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/defaultObjectAcl/{entity}', + request_field=u'objectAccessControl', + request_type_name=u'StorageDefaultObjectAccessControlsUpdateRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + class NotificationsService(base_api.BaseApiService): + """Service class for the notifications resource.""" + + _NAME = u'notifications' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Delete(self, request, global_params=None): + r"""Permanently deletes a notification subscription. + + Args: + request: (StorageNotificationsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageNotificationsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.notifications.delete', + ordered_params=[u'bucket', u'notification'], + path_params=[u'bucket', u'notification'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/notificationConfigs/{notification}', + request_field='', + request_type_name=u'StorageNotificationsDeleteRequest', + response_type_name=u'StorageNotificationsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""View a notification configuration. + + Args: + request: (StorageNotificationsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Notification) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.notifications.get', + ordered_params=[u'bucket', u'notification'], + path_params=[u'bucket', u'notification'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/notificationConfigs/{notification}', + request_field='', + request_type_name=u'StorageNotificationsGetRequest', + response_type_name=u'Notification', + supports_download=False, + ) + + def Insert(self, request, global_params=None): + r"""Creates a notification subscription for a given bucket. + + Args: + request: (StorageNotificationsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Notification) The response message. + """ + config = self.GetMethodConfig('Insert') + return self._RunMethod(config, request, global_params=global_params) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.notifications.insert', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/notificationConfigs', + request_field=u'notification', + request_type_name=u'StorageNotificationsInsertRequest', + response_type_name=u'Notification', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves a list of notification subscriptions for a given bucket. + + Args: + request: (StorageNotificationsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Notifications) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.notifications.list', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[u'userProject'], + relative_path=u'b/{bucket}/notificationConfigs', + request_field='', + request_type_name=u'StorageNotificationsListRequest', + response_type_name=u'Notifications', + supports_download=False, + ) + + class ObjectAccessControlsService(base_api.BaseApiService): + """Service class for the objectAccessControls resource.""" + + _NAME = u'objectAccessControls' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Delete(self, request, global_params=None): + r"""Permanently deletes the ACL entry for the specified entity on the specified object. + + Args: + request: (StorageObjectAccessControlsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageObjectAccessControlsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.objectAccessControls.delete', + ordered_params=[u'bucket', u'object', u'entity'], + path_params=[u'bucket', u'entity', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl/{entity}', + request_field='', + request_type_name=u'StorageObjectAccessControlsDeleteRequest', + response_type_name=u'StorageObjectAccessControlsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns the ACL entry for the specified entity on the specified object. + + Args: + request: (StorageObjectAccessControlsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objectAccessControls.get', + ordered_params=[u'bucket', u'object', u'entity'], + path_params=[u'bucket', u'entity', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl/{entity}', + request_field='', + request_type_name=u'StorageObjectAccessControlsGetRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def Insert(self, request, global_params=None): + r"""Creates a new ACL entry on the specified object. + + Args: + request: (StorageObjectAccessControlsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Insert') + return self._RunMethod(config, request, global_params=global_params) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objectAccessControls.insert', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl', + request_field=u'objectAccessControl', + request_type_name=u'StorageObjectAccessControlsInsertRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves ACL entries on the specified object. + + Args: + request: (StorageObjectAccessControlsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControls) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objectAccessControls.list', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl', + request_field='', + request_type_name=u'StorageObjectAccessControlsListRequest', + response_type_name=u'ObjectAccessControls', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Patches an ACL entry on the specified object. + + Args: + request: (StorageObjectAccessControlsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PATCH', + method_id=u'storage.objectAccessControls.patch', + ordered_params=[u'bucket', u'object', u'entity'], + path_params=[u'bucket', u'entity', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl/{entity}', + request_field=u'objectAccessControl', + request_type_name=u'StorageObjectAccessControlsPatchRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + def Update(self, request, global_params=None): + r"""Updates an ACL entry on the specified object. + + Args: + request: (StorageObjectAccessControlsUpdateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ObjectAccessControl) The response message. + """ + config = self.GetMethodConfig('Update') + return self._RunMethod(config, request, global_params=global_params) + + Update.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.objectAccessControls.update', + ordered_params=[u'bucket', u'object', u'entity'], + path_params=[u'bucket', u'entity', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/acl/{entity}', + request_field=u'objectAccessControl', + request_type_name=u'StorageObjectAccessControlsUpdateRequest', + response_type_name=u'ObjectAccessControl', + supports_download=False, + ) + + class ObjectsService(base_api.BaseApiService): + """Service class for the objects resource.""" + + _NAME = u'objects' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = { + 'Insert': base_api.ApiUploadInfo( + accept=['*/*'], + max_size=None, + resumable_multipart=True, + resumable_path=u'/resumable/upload/storage/v1/b/{bucket}/o', + simple_multipart=True, + simple_path=u'/upload/storage/v1/b/{bucket}/o', + ), + } + + def Compose(self, request, global_params=None): + r"""Concatenates a list of existing objects into a new object in the same bucket. + + Args: + request: (StorageObjectsComposeRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Compose') + return self._RunMethod(config, request, global_params=global_params) + + Compose.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objects.compose', + ordered_params=[u'destinationBucket', u'destinationObject'], + path_params=[u'destinationBucket', u'destinationObject'], + query_params=[ + u'destinationPredefinedAcl', + u'ifGenerationMatch', + u'ifMetagenerationMatch', + u'kmsKeyName', + u'userProject' + ], + relative_path=u'b/{destinationBucket}/o/{destinationObject}/compose', + request_field=u'composeRequest', + request_type_name=u'StorageObjectsComposeRequest', + response_type_name=u'Object', + supports_download=False, + ) + + def Copy(self, request, global_params=None): + r"""Copies a source object to a destination object. Optionally overrides metadata. + + Args: + request: (StorageObjectsCopyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Copy') + return self._RunMethod(config, request, global_params=global_params) + + Copy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objects.copy', + ordered_params=[ + u'sourceBucket', + u'sourceObject', + u'destinationBucket', + u'destinationObject' + ], + path_params=[ + u'destinationBucket', + u'destinationObject', + u'sourceBucket', + u'sourceObject' + ], + query_params=[ + u'destinationPredefinedAcl', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'ifSourceGenerationMatch', + u'ifSourceGenerationNotMatch', + u'ifSourceMetagenerationMatch', + u'ifSourceMetagenerationNotMatch', + u'projection', + u'sourceGeneration', + u'userProject' + ], + relative_path= + u'b/{sourceBucket}/o/{sourceObject}/copyTo/b/{destinationBucket}/o/{destinationObject}', + request_field=u'object', + request_type_name=u'StorageObjectsCopyRequest', + response_type_name=u'Object', + supports_download=False, + ) + + def Delete(self, request, global_params=None): + r"""Deletes an object and its metadata. Deletions are permanent if versioning is not enabled for the bucket, or if the generation parameter is used. + + Args: + request: (StorageObjectsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (StorageObjectsDeleteResponse) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'DELETE', + method_id=u'storage.objects.delete', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[ + u'generation', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'userProject' + ], + relative_path=u'b/{bucket}/o/{object}', + request_field='', + request_type_name=u'StorageObjectsDeleteRequest', + response_type_name=u'StorageObjectsDeleteResponse', + supports_download=False, + ) + + def Get(self, request, global_params=None, download=None): + r"""Retrieves an object or its metadata. + + Args: + request: (StorageObjectsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + download: (Download, default: None) If present, download + data from the request via this stream. + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod( + config, request, global_params=global_params, download=download) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objects.get', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[ + u'generation', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}/o/{object}', + request_field='', + request_type_name=u'StorageObjectsGetRequest', + response_type_name=u'Object', + supports_download=True, + ) + + def GetIamPolicy(self, request, global_params=None): + r"""Returns an IAM policy for the specified object. + + Args: + request: (StorageObjectsGetIamPolicyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Policy) The response message. + """ + config = self.GetMethodConfig('GetIamPolicy') + return self._RunMethod(config, request, global_params=global_params) + + GetIamPolicy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objects.getIamPolicy', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/iam', + request_field='', + request_type_name=u'StorageObjectsGetIamPolicyRequest', + response_type_name=u'Policy', + supports_download=False, + ) + + def Insert(self, request, global_params=None, upload=None): + r"""Stores a new object and metadata. + + Args: + request: (StorageObjectsInsertRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + upload: (Upload, default: None) If present, upload + this stream with the request. + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Insert') + upload_config = self.GetUploadConfig('Insert') + return self._RunMethod( + config, + request, + global_params=global_params, + upload=upload, + upload_config=upload_config) + + Insert.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objects.insert', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'contentEncoding', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'kmsKeyName', + u'name', + u'predefinedAcl', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}/o', + request_field=u'object', + request_type_name=u'StorageObjectsInsertRequest', + response_type_name=u'Object', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Retrieves a list of objects matching the criteria. + + Args: + request: (StorageObjectsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Objects) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objects.list', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'delimiter', + u'includeTrailingDelimiter', + u'maxResults', + u'pageToken', + u'prefix', + u'projection', + u'userProject', + u'versions' + ], + relative_path=u'b/{bucket}/o', + request_field='', + request_type_name=u'StorageObjectsListRequest', + response_type_name=u'Objects', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Patches an object's metadata. + + Args: + request: (StorageObjectsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PATCH', + method_id=u'storage.objects.patch', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[ + u'generation', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'predefinedAcl', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}/o/{object}', + request_field=u'objectResource', + request_type_name=u'StorageObjectsPatchRequest', + response_type_name=u'Object', + supports_download=False, + ) + + def Rewrite(self, request, global_params=None): + r"""Rewrites a source object to a destination object. Optionally overrides metadata. + + Args: + request: (StorageObjectsRewriteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (RewriteResponse) The response message. + """ + config = self.GetMethodConfig('Rewrite') + return self._RunMethod(config, request, global_params=global_params) + + Rewrite.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objects.rewrite', + ordered_params=[ + u'sourceBucket', + u'sourceObject', + u'destinationBucket', + u'destinationObject' + ], + path_params=[ + u'destinationBucket', + u'destinationObject', + u'sourceBucket', + u'sourceObject' + ], + query_params=[ + u'destinationKmsKeyName', + u'destinationPredefinedAcl', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'ifSourceGenerationMatch', + u'ifSourceGenerationNotMatch', + u'ifSourceMetagenerationMatch', + u'ifSourceMetagenerationNotMatch', + u'maxBytesRewrittenPerCall', + u'projection', + u'rewriteToken', + u'sourceGeneration', + u'userProject' + ], + relative_path= + u'b/{sourceBucket}/o/{sourceObject}/rewriteTo/b/{destinationBucket}/o/{destinationObject}', + request_field=u'object', + request_type_name=u'StorageObjectsRewriteRequest', + response_type_name=u'RewriteResponse', + supports_download=False, + ) + + def SetIamPolicy(self, request, global_params=None): + r"""Updates an IAM policy for the specified object. + + Args: + request: (StorageObjectsSetIamPolicyRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Policy) The response message. + """ + config = self.GetMethodConfig('SetIamPolicy') + return self._RunMethod(config, request, global_params=global_params) + + SetIamPolicy.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.objects.setIamPolicy', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[u'generation', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/iam', + request_field=u'policy', + request_type_name=u'StorageObjectsSetIamPolicyRequest', + response_type_name=u'Policy', + supports_download=False, + ) + + def TestIamPermissions(self, request, global_params=None): + r"""Tests a set of permissions on the given object to see which, if any, are held by the caller. + + Args: + request: (StorageObjectsTestIamPermissionsRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (TestIamPermissionsResponse) The response message. + """ + config = self.GetMethodConfig('TestIamPermissions') + return self._RunMethod(config, request, global_params=global_params) + + TestIamPermissions.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.objects.testIamPermissions', + ordered_params=[u'bucket', u'object', u'permissions'], + path_params=[u'bucket', u'object'], + query_params=[u'generation', u'permissions', u'userProject'], + relative_path=u'b/{bucket}/o/{object}/iam/testPermissions', + request_field='', + request_type_name=u'StorageObjectsTestIamPermissionsRequest', + response_type_name=u'TestIamPermissionsResponse', + supports_download=False, + ) + + def Update(self, request, global_params=None): + r"""Updates an object's metadata. + + Args: + request: (StorageObjectsUpdateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Object) The response message. + """ + config = self.GetMethodConfig('Update') + return self._RunMethod(config, request, global_params=global_params) + + Update.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'storage.objects.update', + ordered_params=[u'bucket', u'object'], + path_params=[u'bucket', u'object'], + query_params=[ + u'generation', + u'ifGenerationMatch', + u'ifGenerationNotMatch', + u'ifMetagenerationMatch', + u'ifMetagenerationNotMatch', + u'predefinedAcl', + u'projection', + u'userProject' + ], + relative_path=u'b/{bucket}/o/{object}', + request_field=u'objectResource', + request_type_name=u'StorageObjectsUpdateRequest', + response_type_name=u'Object', + supports_download=False, + ) + + def WatchAll(self, request, global_params=None): + r"""Watch for changes on all objects in a bucket. + + Args: + request: (StorageObjectsWatchAllRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Channel) The response message. + """ + config = self.GetMethodConfig('WatchAll') + return self._RunMethod(config, request, global_params=global_params) + + WatchAll.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'storage.objects.watchAll', + ordered_params=[u'bucket'], + path_params=[u'bucket'], + query_params=[ + u'delimiter', + u'includeTrailingDelimiter', + u'maxResults', + u'pageToken', + u'prefix', + u'projection', + u'userProject', + u'versions' + ], + relative_path=u'b/{bucket}/o/watch', + request_field=u'channel', + request_type_name=u'StorageObjectsWatchAllRequest', + response_type_name=u'Channel', + supports_download=False, + ) + + class ProjectsServiceAccountService(base_api.BaseApiService): + """Service class for the projects_serviceAccount resource.""" + + _NAME = u'projects_serviceAccount' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} + + def Get(self, request, global_params=None): + r"""Get the email address of this project's Google Cloud Storage service account. + + Args: + request: (StorageProjectsServiceAccountGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ServiceAccount) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'storage.projects.serviceAccount.get', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'userProject'], + relative_path=u'projects/{projectId}/serviceAccount', + request_field='', + request_type_name=u'StorageProjectsServiceAccountGetRequest', + response_type_name=u'ServiceAccount', + supports_download=False, + ) + + class ProjectsService(base_api.BaseApiService): + """Service class for the projects resource.""" + + _NAME = u'projects' + + def __init__(self, client): + super().__init__(client) + self._upload_configs = {} diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py new file mode 100644 index 000000000000..caef0eb4b033 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py @@ -0,0 +1,2714 @@ +# +# 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. +# + +"""Generated message classes for storage version v1. + +Stores and retrieves potentially large, immutable data objects. +""" +# NOTE: This file is autogenerated and should not be edited by hand. + +from apitools.base.protorpclite import message_types as _message_types +from apitools.base.protorpclite import messages as _messages +from apitools.base.py import encoding, extra_types + +package = 'storage' + + +class Bucket(_messages.Message): + r"""A bucket. + + Messages: + BillingValue: The bucket's billing configuration. + CorsValueListEntry: A CorsValueListEntry object. + EncryptionValue: Encryption configuration for a bucket. + LabelsValue: User-provided labels, in key/value pairs. + LifecycleValue: The bucket's lifecycle configuration. See lifecycle + management for more information. + LoggingValue: The bucket's logging configuration, which defines the + destination bucket and optional name prefix for the current bucket's + logs. + OwnerValue: The owner of the bucket. This is always the project team's + owner group. + RetentionPolicyValue: The bucket's retention policy. The retention policy + enforces a minimum retention time for all objects contained in the + bucket, based on their creation time. Any attempt to overwrite or delete + objects younger than the retention period will result in a + PERMISSION_DENIED error. An unlocked retention policy can be modified or + removed from the bucket via a storage.buckets.update operation. A locked + retention policy cannot be removed or shortened in duration for the + lifetime of the bucket. Attempting to remove or decrease period of a + locked retention policy will result in a PERMISSION_DENIED error. + VersioningValue: The bucket's versioning configuration. + WebsiteValue: The bucket's website configuration, controlling how the + service behaves when accessing bucket contents as a web site. See the + Static Website Examples for more information. + + Fields: + acl: Access controls on the bucket. + billing: The bucket's billing configuration. + cors: The bucket's Cross-Origin Resource Sharing (CORS) configuration. + defaultEventBasedHold: The default value for event-based hold on newly + created objects in this bucket. Event-based hold is a way to retain + objects indefinitely until an event occurs, signified by the hold's + release. After being released, such objects will be subject to bucket- + level retention (if any). One sample use case of this flag is for banks + to hold loan documents for at least 3 years after loan is paid in full. + Here, bucket-level retention is 3 years and the event is loan being paid + in full. In this example, these objects will be held intact for any + number of years until the event has occurred (event-based hold on the + object is released) and then 3 more years after that. That means + retention duration of the objects begins from the moment event-based + hold transitioned from true to false. Objects under event-based hold + cannot be deleted, overwritten or archived until the hold is removed. + defaultObjectAcl: Default access controls to apply to new objects when no + ACL is provided. + encryption: Encryption configuration for a bucket. + etag: HTTP 1.1 Entity tag for the bucket. + id: The ID of the bucket. For buckets, the id and name properties are the + same. + kind: The kind of item this is. For buckets, this is always + storage#bucket. + labels: User-provided labels, in key/value pairs. + lifecycle: The bucket's lifecycle configuration. See lifecycle management + for more information. + location: The location of the bucket. Object data for objects in the + bucket resides in physical storage within this region. Defaults to US. + See the developer's guide for the authoritative list. + logging: The bucket's logging configuration, which defines the destination + bucket and optional name prefix for the current bucket's logs. + metageneration: The metadata generation of this bucket. + name: The name of the bucket. + owner: The owner of the bucket. This is always the project team's owner + group. + projectNumber: The project number of the project the bucket belongs to. + retentionPolicy: The bucket's retention policy. The retention policy + enforces a minimum retention time for all objects contained in the + bucket, based on their creation time. Any attempt to overwrite or delete + objects younger than the retention period will result in a + PERMISSION_DENIED error. An unlocked retention policy can be modified or + removed from the bucket via a storage.buckets.update operation. A locked + retention policy cannot be removed or shortened in duration for the + lifetime of the bucket. Attempting to remove or decrease period of a + locked retention policy will result in a PERMISSION_DENIED error. + selfLink: The URI of this bucket. + storageClass: The bucket's default storage class, used whenever no + storageClass is specified for a newly-created object. This defines how + objects in the bucket are stored and determines the SLA and the cost of + storage. Values include MULTI_REGIONAL, REGIONAL, STANDARD, NEARLINE, + COLDLINE, and DURABLE_REDUCED_AVAILABILITY. If this value is not + specified when the bucket is created, it will default to STANDARD. For + more information, see storage classes. + timeCreated: The creation time of the bucket in RFC 3339 format. + updated: The modification time of the bucket in RFC 3339 format. + versioning: The bucket's versioning configuration. + website: The bucket's website configuration, controlling how the service + behaves when accessing bucket contents as a web site. See the Static + Website Examples for more information. + """ + class BillingValue(_messages.Message): + r"""The bucket's billing configuration. + + Fields: + requesterPays: When set to true, Requester Pays is enabled for this + bucket. + """ + + requesterPays = _messages.BooleanField(1) + + class CorsValueListEntry(_messages.Message): + r"""A CorsValueListEntry object. + + Fields: + maxAgeSeconds: The value, in seconds, to return in the Access-Control- + Max-Age header used in preflight responses. + method: The list of HTTP methods on which to include CORS response + headers, (GET, OPTIONS, POST, etc) Note: "*" is permitted in the list + of methods, and means "any method". + origin: The list of Origins eligible to receive CORS response headers. + Note: "*" is permitted in the list of origins, and means "any Origin". + responseHeader: The list of HTTP headers other than the simple response + headers to give permission for the user-agent to share across domains. + """ + + maxAgeSeconds = _messages.IntegerField(1, variant=_messages.Variant.INT32) + method = _messages.StringField(2, repeated=True) + origin = _messages.StringField(3, repeated=True) + responseHeader = _messages.StringField(4, repeated=True) + + class EncryptionValue(_messages.Message): + r"""Encryption configuration for a bucket. + + Fields: + defaultKmsKeyName: A Cloud KMS key that will be used to encrypt objects + inserted into this bucket, if no encryption method is specified. + """ + + defaultKmsKeyName = _messages.StringField(1) + + @encoding.MapUnrecognizedFields('additionalProperties') + class LabelsValue(_messages.Message): + r"""User-provided labels, in key/value pairs. + + Messages: + AdditionalProperty: An additional property for a LabelsValue object. + + Fields: + additionalProperties: An individual label entry. + """ + class AdditionalProperty(_messages.Message): + r"""An additional property for a LabelsValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField( + 'AdditionalProperty', 1, repeated=True) + + class LifecycleValue(_messages.Message): + r"""The bucket's lifecycle configuration. See lifecycle management for + more information. + + Messages: + RuleValueListEntry: A RuleValueListEntry object. + + Fields: + rule: A lifecycle management rule, which is made of an action to take + and the condition(s) under which the action will be taken. + """ + class RuleValueListEntry(_messages.Message): + r"""A RuleValueListEntry object. + + Messages: + ActionValue: The action to take. + ConditionValue: The condition(s) under which the action will be taken. + + Fields: + action: The action to take. + condition: The condition(s) under which the action will be taken. + """ + class ActionValue(_messages.Message): + r"""The action to take. + + Fields: + storageClass: Target storage class. Required iff the type of the + action is SetStorageClass. + type: Type of the action. Currently, only Delete and SetStorageClass + are supported. + """ + + storageClass = _messages.StringField(1) + type = _messages.StringField(2) + + class ConditionValue(_messages.Message): + r"""The condition(s) under which the action will be taken. + + Fields: + age: Age of an object (in days). This condition is satisfied when an + object reaches the specified age. + createdBefore: A date in RFC 3339 format with only the date part + (for instance, "2013-01-15"). This condition is satisfied when an + object is created before midnight of the specified date in UTC. + isLive: Relevant only for versioned objects. If the value is true, + this condition matches live objects; if the value is false, it + matches archived objects. + matchesPattern: A regular expression that satisfies the RE2 syntax. + This condition is satisfied when the name of the object matches + the RE2 pattern. Note: This feature is currently in the "Early + Access" launch stage and is only available to a whitelisted set of + users; that means that this feature may be changed in backward- + incompatible ways and that it is not guaranteed to be released. + matchesStorageClass: Objects having any of the storage classes + specified by this condition will be matched. Values include + MULTI_REGIONAL, REGIONAL, NEARLINE, COLDLINE, STANDARD, and + DURABLE_REDUCED_AVAILABILITY. + numNewerVersions: Relevant only for versioned objects. If the value + is N, this condition is satisfied when there are at least N + versions (including the live version) newer than this version of + the object. + """ + + age = _messages.IntegerField(1, variant=_messages.Variant.INT32) + createdBefore = extra_types.DateField(2) + isLive = _messages.BooleanField(3) + matchesPattern = _messages.StringField(4) + matchesStorageClass = _messages.StringField(5, repeated=True) + numNewerVersions = _messages.IntegerField( + 6, variant=_messages.Variant.INT32) + + action = _messages.MessageField('ActionValue', 1) + condition = _messages.MessageField('ConditionValue', 2) + + rule = _messages.MessageField('RuleValueListEntry', 1, repeated=True) + + class LoggingValue(_messages.Message): + r"""The bucket's logging configuration, which defines the destination + bucket and optional name prefix for the current bucket's logs. + + Fields: + logBucket: The destination bucket where the current bucket's logs should + be placed. + logObjectPrefix: A prefix for log object names. + """ + + logBucket = _messages.StringField(1) + logObjectPrefix = _messages.StringField(2) + + class OwnerValue(_messages.Message): + r"""The owner of the bucket. This is always the project team's owner + group. + + Fields: + entity: The entity, in the form project-owner-projectId. + entityId: The ID for the entity. + """ + + entity = _messages.StringField(1) + entityId = _messages.StringField(2) + + class RetentionPolicyValue(_messages.Message): + r"""The bucket's retention policy. The retention policy enforces a minimum + retention time for all objects contained in the bucket, based on their + creation time. Any attempt to overwrite or delete objects younger than the + retention period will result in a PERMISSION_DENIED error. An unlocked + retention policy can be modified or removed from the bucket via a + storage.buckets.update operation. A locked retention policy cannot be + removed or shortened in duration for the lifetime of the bucket. + Attempting to remove or decrease period of a locked retention policy will + result in a PERMISSION_DENIED error. + + Fields: + effectiveTime: Server-determined value that indicates the time from + which policy was enforced and effective. This value is in RFC 3339 + format. + isLocked: Once locked, an object retention policy cannot be modified. + retentionPeriod: The duration in seconds that objects need to be + retained. Retention duration must be greater than zero and less than + 100 years. Note that enforcement of retention periods less than a day + is not guaranteed. Such periods should only be used for testing + purposes. + """ + + effectiveTime = _message_types.DateTimeField(1) + isLocked = _messages.BooleanField(2) + retentionPeriod = _messages.IntegerField(3) + + class VersioningValue(_messages.Message): + r"""The bucket's versioning configuration. + + Fields: + enabled: While set to true, versioning is fully enabled for this bucket. + """ + + enabled = _messages.BooleanField(1) + + class WebsiteValue(_messages.Message): + r"""The bucket's website configuration, controlling how the service + behaves when accessing bucket contents as a web site. See the Static + Website Examples for more information. + + Fields: + mainPageSuffix: If the requested object path is missing, the service + will ensure the path has a trailing '/', append this suffix, and + attempt to retrieve the resulting object. This allows the creation of + index.html objects to represent directory pages. + notFoundPage: If the requested object path is missing, and any + mainPageSuffix object is missing, if applicable, the service will + return the named object from this bucket as the content for a 404 Not + Found result. + """ + + mainPageSuffix = _messages.StringField(1) + notFoundPage = _messages.StringField(2) + + acl = _messages.MessageField('BucketAccessControl', 1, repeated=True) + billing = _messages.MessageField('BillingValue', 2) + cors = _messages.MessageField('CorsValueListEntry', 3, repeated=True) + defaultEventBasedHold = _messages.BooleanField(4) + defaultObjectAcl = _messages.MessageField( + 'ObjectAccessControl', 5, repeated=True) + encryption = _messages.MessageField('EncryptionValue', 6) + etag = _messages.StringField(7) + id = _messages.StringField(8) + kind = _messages.StringField(9, default=u'storage#bucket') + labels = _messages.MessageField('LabelsValue', 10) + lifecycle = _messages.MessageField('LifecycleValue', 11) + location = _messages.StringField(12) + logging = _messages.MessageField('LoggingValue', 13) + metageneration = _messages.IntegerField(14) + name = _messages.StringField(15) + owner = _messages.MessageField('OwnerValue', 16) + projectNumber = _messages.IntegerField(17, variant=_messages.Variant.UINT64) + retentionPolicy = _messages.MessageField('RetentionPolicyValue', 18) + selfLink = _messages.StringField(19) + storageClass = _messages.StringField(20) + timeCreated = _message_types.DateTimeField(21) + updated = _message_types.DateTimeField(22) + versioning = _messages.MessageField('VersioningValue', 23) + website = _messages.MessageField('WebsiteValue', 24) + + +class BucketAccessControl(_messages.Message): + r"""An access-control entry. + + Messages: + ProjectTeamValue: The project team associated with the entity, if any. + + Fields: + bucket: The name of the bucket. + domain: The domain associated with the entity, if any. + email: The email address associated with the entity, if any. + entity: The entity holding the permission, in one of the following forms: + - user-userId - user-email - group-groupId - group-email - domain- + domain - project-team-projectId - allUsers - allAuthenticatedUsers + Examples: - The user liz@example.com would be user-liz@example.com. - + The group example@googlegroups.com would be group- + example@googlegroups.com. - To refer to all members of the Google Apps + for Business domain example.com, the entity would be domain-example.com. + entityId: The ID for the entity, if any. + etag: HTTP 1.1 Entity tag for the access-control entry. + id: The ID of the access-control entry. + kind: The kind of item this is. For bucket access control entries, this is + always storage#bucketAccessControl. + projectTeam: The project team associated with the entity, if any. + role: The access permission for the entity. + selfLink: The link to this access-control entry. + """ + class ProjectTeamValue(_messages.Message): + r"""The project team associated with the entity, if any. + + Fields: + projectNumber: The project number. + team: The team. + """ + + projectNumber = _messages.StringField(1) + team = _messages.StringField(2) + + bucket = _messages.StringField(1) + domain = _messages.StringField(2) + email = _messages.StringField(3) + entity = _messages.StringField(4) + entityId = _messages.StringField(5) + etag = _messages.StringField(6) + id = _messages.StringField(7) + kind = _messages.StringField(8, default=u'storage#bucketAccessControl') + projectTeam = _messages.MessageField('ProjectTeamValue', 9) + role = _messages.StringField(10) + selfLink = _messages.StringField(11) + + +class BucketAccessControls(_messages.Message): + r"""An access-control list. + + Fields: + items: The list of items. + kind: The kind of item this is. For lists of bucket access control + entries, this is always storage#bucketAccessControls. + """ + + items = _messages.MessageField('BucketAccessControl', 1, repeated=True) + kind = _messages.StringField(2, default=u'storage#bucketAccessControls') + + +class Buckets(_messages.Message): + r"""A list of buckets. + + Fields: + items: The list of items. + kind: The kind of item this is. For lists of buckets, this is always + storage#buckets. + nextPageToken: The continuation token, used to page through large result + sets. Provide this value in a subsequent request to return the next page + of results. + """ + + items = _messages.MessageField('Bucket', 1, repeated=True) + kind = _messages.StringField(2, default=u'storage#buckets') + nextPageToken = _messages.StringField(3) + + +class Channel(_messages.Message): + r"""An notification channel used to watch for resource changes. + + Messages: + ParamsValue: Additional parameters controlling delivery channel behavior. + Optional. + + Fields: + address: The address where notifications are delivered for this channel. + expiration: Date and time of notification channel expiration, expressed as + a Unix timestamp, in milliseconds. Optional. + id: A UUID or similar unique string that identifies this channel. + kind: Identifies this as a notification channel used to watch for changes + to a resource. Value: the fixed string "api#channel". + params: Additional parameters controlling delivery channel behavior. + Optional. + payload: A Boolean value to indicate whether payload is wanted. Optional. + resourceId: An opaque ID that identifies the resource being watched on + this channel. Stable across different API versions. + resourceUri: A version-specific identifier for the watched resource. + token: An arbitrary string delivered to the target address with each + notification delivered over this channel. Optional. + type: The type of delivery mechanism used for this channel. + """ + @encoding.MapUnrecognizedFields('additionalProperties') + class ParamsValue(_messages.Message): + r"""Additional parameters controlling delivery channel behavior. Optional. + + Messages: + AdditionalProperty: An additional property for a ParamsValue object. + + Fields: + additionalProperties: Declares a new parameter by name. + """ + class AdditionalProperty(_messages.Message): + r"""An additional property for a ParamsValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField( + 'AdditionalProperty', 1, repeated=True) + + address = _messages.StringField(1) + expiration = _messages.IntegerField(2) + id = _messages.StringField(3) + kind = _messages.StringField(4, default=u'api#channel') + params = _messages.MessageField('ParamsValue', 5) + payload = _messages.BooleanField(6) + resourceId = _messages.StringField(7) + resourceUri = _messages.StringField(8) + token = _messages.StringField(9) + type = _messages.StringField(10) + + +class ComposeRequest(_messages.Message): + r"""A Compose request. + + Messages: + SourceObjectsValueListEntry: A SourceObjectsValueListEntry object. + + Fields: + destination: Properties of the resulting object. + kind: The kind of item this is. + sourceObjects: The list of source objects that will be concatenated into a + single object. + """ + class SourceObjectsValueListEntry(_messages.Message): + r"""A SourceObjectsValueListEntry object. + + Messages: + ObjectPreconditionsValue: Conditions that must be met for this operation + to execute. + + Fields: + generation: The generation of this object to use as the source. + name: The source object's name. All source objects must reside in the + same bucket. + objectPreconditions: Conditions that must be met for this operation to + execute. + """ + class ObjectPreconditionsValue(_messages.Message): + r"""Conditions that must be met for this operation to execute. + + Fields: + ifGenerationMatch: Only perform the composition if the generation of + the source object that would be used matches this value. If this + value and a generation are both specified, they must be the same + value or the call will fail. + """ + + ifGenerationMatch = _messages.IntegerField(1) + + generation = _messages.IntegerField(1) + name = _messages.StringField(2) + objectPreconditions = _messages.MessageField('ObjectPreconditionsValue', 3) + + destination = _messages.MessageField('Object', 1) + kind = _messages.StringField(2, default=u'storage#composeRequest') + sourceObjects = _messages.MessageField( + 'SourceObjectsValueListEntry', 3, repeated=True) + + +class Notification(_messages.Message): + r"""A subscription to receive Google PubSub notifications. + + Messages: + CustomAttributesValue: An optional list of additional attributes to attach + to each Cloud PubSub message published for this notification + subscription. + + Fields: + custom_attributes: An optional list of additional attributes to attach to + each Cloud PubSub message published for this notification subscription. + etag: HTTP 1.1 Entity tag for this subscription notification. + event_types: If present, only send notifications about listed event types. + If empty, sent notifications for all event types. + id: The ID of the notification. + kind: The kind of item this is. For notifications, this is always + storage#notification. + object_name_prefix: If present, only apply this notification configuration + to object names that begin with this prefix. + payload_format: The desired content of the Payload. + selfLink: The canonical URL of this notification. + topic: The Cloud PubSub topic to which this subscription publishes. + Formatted as: '//pubsub.googleapis.com/projects/{project- + identifier}/topics/{my-topic}' + """ + @encoding.MapUnrecognizedFields('additionalProperties') + class CustomAttributesValue(_messages.Message): + r"""An optional list of additional attributes to attach to each Cloud + PubSub message published for this notification subscription. + + Messages: + AdditionalProperty: An additional property for a CustomAttributesValue + object. + + Fields: + additionalProperties: Additional properties of type + CustomAttributesValue + """ + class AdditionalProperty(_messages.Message): + r"""An additional property for a CustomAttributesValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField( + 'AdditionalProperty', 1, repeated=True) + + custom_attributes = _messages.MessageField('CustomAttributesValue', 1) + etag = _messages.StringField(2) + event_types = _messages.StringField(3, repeated=True) + id = _messages.StringField(4) + kind = _messages.StringField(5, default=u'storage#notification') + object_name_prefix = _messages.StringField(6) + payload_format = _messages.StringField(7, default=u'JSON_API_V1') + selfLink = _messages.StringField(8) + topic = _messages.StringField(9) + + +class Notifications(_messages.Message): + r"""A list of notification subscriptions. + + Fields: + items: The list of items. + kind: The kind of item this is. For lists of notifications, this is always + storage#notifications. + """ + + items = _messages.MessageField('Notification', 1, repeated=True) + kind = _messages.StringField(2, default=u'storage#notifications') + + +class Object(_messages.Message): + r"""An object. + + Messages: + CustomerEncryptionValue: Metadata of customer-supplied encryption key, if + the object is encrypted by such a key. + MetadataValue: User-provided metadata, in key/value pairs. + OwnerValue: The owner of the object. This will always be the uploader of + the object. + + Fields: + acl: Access controls on the object. + bucket: The name of the bucket containing this object. + cacheControl: Cache-Control directive for the object data. If omitted, and + the object is accessible to all anonymous users, the default will be + public, max-age=3600. + componentCount: Number of underlying components that make up this object. + Components are accumulated by compose operations. + contentDisposition: Content-Disposition of the object data. + contentEncoding: Content-Encoding of the object data. + contentLanguage: Content-Language of the object data. + contentType: Content-Type of the object data. If an object is stored + without a Content-Type, it is served as application/octet-stream. + crc32c: CRC32c checksum, as described in RFC 4960, Appendix B; encoded + using base64 in big-endian byte order. For more information about using + the CRC32c checksum, see Hashes and ETags: Best Practices. + customerEncryption: Metadata of customer-supplied encryption key, if the + object is encrypted by such a key. + etag: HTTP 1.1 Entity tag for the object. + eventBasedHold: Whether an object is under event-based hold. Event-based + hold is a way to retain objects until an event occurs, which is + signified by the hold's release (i.e. this value is set to false). After + being released (set to false), such objects will be subject to bucket- + level retention (if any). One sample use case of this flag is for banks + to hold loan documents for at least 3 years after loan is paid in full. + Here, bucket-level retention is 3 years and the event is the loan being + paid in full. In this example, these objects will be held intact for any + number of years until the event has occurred (event-based hold on the + object is released) and then 3 more years after that. That means + retention duration of the objects begins from the moment event-based + hold transitioned from true to false. + generation: The content generation of this object. Used for object + versioning. + id: The ID of the object, including the bucket name, object name, and + generation number. + kind: The kind of item this is. For objects, this is always + storage#object. + kmsKeyName: Cloud KMS Key used to encrypt this object, if the object is + encrypted by such a key. + md5Hash: MD5 hash of the data; encoded using base64. For more information + about using the MD5 hash, see Hashes and ETags: Best Practices. + mediaLink: Media download link. + metadata: User-provided metadata, in key/value pairs. + metageneration: The version of the metadata for this object at this + generation. Used for preconditions and for detecting changes in + metadata. A metageneration number is only meaningful in the context of a + particular generation of a particular object. + name: The name of the object. Required if not specified by URL parameter. + owner: The owner of the object. This will always be the uploader of the + object. + retentionExpirationTime: A server-determined value that specifies the + earliest time that the object's retention period expires. This value is + in RFC 3339 format. Note 1: This field is not provided for objects with + an active event-based hold, since retention expiration is unknown until + the hold is removed. Note 2: This value can be provided even when + temporary hold is set (so that the user can reason about policy without + having to first unset the temporary hold). + selfLink: The link to this object. + size: Content-Length of the data in bytes. + storageClass: Storage class of the object. + temporaryHold: Whether an object is under temporary hold. While this flag + is set to true, the object is protected against deletion and overwrites. + A common use case of this flag is regulatory investigations where + objects need to be retained while the investigation is ongoing. Note + that unlike event-based hold, temporary hold does not impact retention + expiration time of an object. + timeCreated: The creation time of the object in RFC 3339 format. + timeDeleted: The deletion time of the object in RFC 3339 format. Will be + returned if and only if this version of the object has been deleted. + timeStorageClassUpdated: The time at which the object's storage class was + last changed. When the object is initially created, it will be set to + timeCreated. + updated: The modification time of the object metadata in RFC 3339 format. + """ + class CustomerEncryptionValue(_messages.Message): + r"""Metadata of customer-supplied encryption key, if the object is + encrypted by such a key. + + Fields: + encryptionAlgorithm: The encryption algorithm. + keySha256: SHA256 hash value of the encryption key. + """ + + encryptionAlgorithm = _messages.StringField(1) + keySha256 = _messages.StringField(2) + + @encoding.MapUnrecognizedFields('additionalProperties') + class MetadataValue(_messages.Message): + r"""User-provided metadata, in key/value pairs. + + Messages: + AdditionalProperty: An additional property for a MetadataValue object. + + Fields: + additionalProperties: An individual metadata entry. + """ + class AdditionalProperty(_messages.Message): + r"""An additional property for a MetadataValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField( + 'AdditionalProperty', 1, repeated=True) + + class OwnerValue(_messages.Message): + r"""The owner of the object. This will always be the uploader of the + object. + + Fields: + entity: The entity, in the form user-userId. + entityId: The ID for the entity. + """ + + entity = _messages.StringField(1) + entityId = _messages.StringField(2) + + acl = _messages.MessageField('ObjectAccessControl', 1, repeated=True) + bucket = _messages.StringField(2) + cacheControl = _messages.StringField(3) + componentCount = _messages.IntegerField(4, variant=_messages.Variant.INT32) + contentDisposition = _messages.StringField(5) + contentEncoding = _messages.StringField(6) + contentLanguage = _messages.StringField(7) + contentType = _messages.StringField(8) + crc32c = _messages.StringField(9) + customerEncryption = _messages.MessageField('CustomerEncryptionValue', 10) + etag = _messages.StringField(11) + eventBasedHold = _messages.BooleanField(12) + generation = _messages.IntegerField(13) + id = _messages.StringField(14) + kind = _messages.StringField(15, default=u'storage#object') + kmsKeyName = _messages.StringField(16) + md5Hash = _messages.StringField(17) + mediaLink = _messages.StringField(18) + metadata = _messages.MessageField('MetadataValue', 19) + metageneration = _messages.IntegerField(20) + name = _messages.StringField(21) + owner = _messages.MessageField('OwnerValue', 22) + retentionExpirationTime = _message_types.DateTimeField(23) + selfLink = _messages.StringField(24) + size = _messages.IntegerField(25, variant=_messages.Variant.UINT64) + storageClass = _messages.StringField(26) + temporaryHold = _messages.BooleanField(27) + timeCreated = _message_types.DateTimeField(28) + timeDeleted = _message_types.DateTimeField(29) + timeStorageClassUpdated = _message_types.DateTimeField(30) + updated = _message_types.DateTimeField(31) + + +class ObjectAccessControl(_messages.Message): + r"""An access-control entry. + + Messages: + ProjectTeamValue: The project team associated with the entity, if any. + + Fields: + bucket: The name of the bucket. + domain: The domain associated with the entity, if any. + email: The email address associated with the entity, if any. + entity: The entity holding the permission, in one of the following forms: + - user-userId - user-email - group-groupId - group-email - domain- + domain - project-team-projectId - allUsers - allAuthenticatedUsers + Examples: - The user liz@example.com would be user-liz@example.com. - + The group example@googlegroups.com would be group- + example@googlegroups.com. - To refer to all members of the Google Apps + for Business domain example.com, the entity would be domain-example.com. + entityId: The ID for the entity, if any. + etag: HTTP 1.1 Entity tag for the access-control entry. + generation: The content generation of the object, if applied to an object. + id: The ID of the access-control entry. + kind: The kind of item this is. For object access control entries, this is + always storage#objectAccessControl. + object: The name of the object, if applied to an object. + projectTeam: The project team associated with the entity, if any. + role: The access permission for the entity. + selfLink: The link to this access-control entry. + """ + class ProjectTeamValue(_messages.Message): + r"""The project team associated with the entity, if any. + + Fields: + projectNumber: The project number. + team: The team. + """ + + projectNumber = _messages.StringField(1) + team = _messages.StringField(2) + + bucket = _messages.StringField(1) + domain = _messages.StringField(2) + email = _messages.StringField(3) + entity = _messages.StringField(4) + entityId = _messages.StringField(5) + etag = _messages.StringField(6) + generation = _messages.IntegerField(7) + id = _messages.StringField(8) + kind = _messages.StringField(9, default=u'storage#objectAccessControl') + object = _messages.StringField(10) + projectTeam = _messages.MessageField('ProjectTeamValue', 11) + role = _messages.StringField(12) + selfLink = _messages.StringField(13) + + +class ObjectAccessControls(_messages.Message): + r"""An access-control list. + + Fields: + items: The list of items. + kind: The kind of item this is. For lists of object access control + entries, this is always storage#objectAccessControls. + """ + + items = _messages.MessageField('ObjectAccessControl', 1, repeated=True) + kind = _messages.StringField(2, default=u'storage#objectAccessControls') + + +class Objects(_messages.Message): + r"""A list of objects. + + Fields: + items: The list of items. + kind: The kind of item this is. For lists of objects, this is always + storage#objects. + nextPageToken: The continuation token, used to page through large result + sets. Provide this value in a subsequent request to return the next page + of results. + prefixes: The list of prefixes of objects matching-but-not-listed up to + and including the requested delimiter. + """ + + items = _messages.MessageField('Object', 1, repeated=True) + kind = _messages.StringField(2, default=u'storage#objects') + nextPageToken = _messages.StringField(3) + prefixes = _messages.StringField(4, repeated=True) + + +class Policy(_messages.Message): + r"""A bucket/object IAM policy. + + Messages: + BindingsValueListEntry: A BindingsValueListEntry object. + + Fields: + bindings: An association between a role, which comes with a set of + permissions, and members who may assume that role. + etag: HTTP 1.1 Entity tag for the policy. + kind: The kind of item this is. For policies, this is always + storage#policy. This field is ignored on input. + resourceId: The ID of the resource to which this policy belongs. Will be + of the form projects/_/buckets/bucket for buckets, and + projects/_/buckets/bucket/objects/object for objects. A specific + generation may be specified by appending #generationNumber to the end of + the object name, e.g. projects/_/buckets/my-bucket/objects/data.txt#17. + The current generation can be denoted with #0. This field is ignored on + input. + """ + class BindingsValueListEntry(_messages.Message): + r"""A BindingsValueListEntry object. + + Fields: + condition: A extra_types.JsonValue attribute. + members: A collection of identifiers for members who may assume the + provided role. Recognized identifiers are as follows: - allUsers - A + special identifier that represents anyone on the internet; with or + without a Google account. - allAuthenticatedUsers - A special + identifier that represents anyone who is authenticated with a Google + account or a service account. - user:emailid - An email address that + represents a specific account. For example, user:alice@gmail.com or + user:joe@example.com. - serviceAccount:emailid - An email address + that represents a service account. For example, serviceAccount:my- + other-app@appspot.gserviceaccount.com . - group:emailid - An email + address that represents a Google group. For example, + group:admins@example.com. - domain:domain - A Google Apps domain + name that represents all the users of that domain. For example, + domain:google.com or domain:example.com. - projectOwner:projectid - + Owners of the given project. For example, projectOwner:my-example- + project - projectEditor:projectid - Editors of the given project. + For example, projectEditor:my-example-project - + projectViewer:projectid - Viewers of the given project. For example, + projectViewer:my-example-project + role: The role to which members belong. Two types of roles are + supported: new IAM roles, which grant permissions that do not map + directly to those provided by ACLs, and legacy IAM roles, which do map + directly to ACL permissions. All roles are of the format + roles/storage.specificRole. The new IAM roles are: - + roles/storage.admin - Full control of Google Cloud Storage resources. + - roles/storage.objectViewer - Read-Only access to Google Cloud + Storage objects. - roles/storage.objectCreator - Access to create + objects in Google Cloud Storage. - roles/storage.objectAdmin - Full + control of Google Cloud Storage objects. The legacy IAM roles are: + - roles/storage.legacyObjectReader - Read-only access to objects + without listing. Equivalent to an ACL entry on an object with the + READER role. - roles/storage.legacyObjectOwner - Read/write access + to existing objects without listing. Equivalent to an ACL entry on an + object with the OWNER role. - roles/storage.legacyBucketReader - + Read access to buckets with object listing. Equivalent to an ACL entry + on a bucket with the READER role. - roles/storage.legacyBucketWriter + - Read access to buckets with object listing/creation/deletion. + Equivalent to an ACL entry on a bucket with the WRITER role. - + roles/storage.legacyBucketOwner - Read and write access to existing + buckets with object listing/creation/deletion. Equivalent to an ACL + entry on a bucket with the OWNER role. + """ + + condition = _messages.MessageField('extra_types.JsonValue', 1) + members = _messages.StringField(2, repeated=True) + role = _messages.StringField(3) + + bindings = _messages.MessageField('BindingsValueListEntry', 1, repeated=True) + etag = _messages.BytesField(2) + kind = _messages.StringField(3, default=u'storage#policy') + resourceId = _messages.StringField(4) + + +class RewriteResponse(_messages.Message): + r"""A rewrite response. + + Fields: + done: true if the copy is finished; otherwise, false if the copy is in + progress. This property is always present in the response. + kind: The kind of item this is. + objectSize: The total size of the object being copied in bytes. This + property is always present in the response. + resource: A resource containing the metadata for the copied-to object. + This property is present in the response only when copying completes. + rewriteToken: A token to use in subsequent requests to continue copying + data. This token is present in the response only when there is more data + to copy. + totalBytesRewritten: The total bytes written so far, which can be used to + provide a waiting user with a progress indicator. This property is + always present in the response. + """ + + done = _messages.BooleanField(1) + kind = _messages.StringField(2, default=u'storage#rewriteResponse') + objectSize = _messages.IntegerField(3) + resource = _messages.MessageField('Object', 4) + rewriteToken = _messages.StringField(5) + totalBytesRewritten = _messages.IntegerField(6) + + +class ServiceAccount(_messages.Message): + r"""A subscription to receive Google PubSub notifications. + + Fields: + email_address: The ID of the notification. + kind: The kind of item this is. For notifications, this is always + storage#notification. + """ + + email_address = _messages.StringField(1) + kind = _messages.StringField(2, default=u'storage#serviceAccount') + + +class StandardQueryParameters(_messages.Message): + r"""Query parameters accepted by all methods. + + Enums: + AltValueValuesEnum: Data format for the response. + + Fields: + alt: Data format for the response. + fields: Selector specifying which fields to include in a partial response. + key: API key. Your API key identifies your project and provides you with + API access, quota, and reports. Required unless you provide an OAuth 2.0 + token. + oauth_token: OAuth 2.0 token for the current user. + prettyPrint: Returns response with indentations and line breaks. + quotaUser: An opaque string that represents a user for quota purposes. + Must not exceed 40 characters. + trace: A tracing token of the form "token:" to include in api + requests. + userIp: Deprecated. Please use quotaUser instead. + """ + class AltValueValuesEnum(_messages.Enum): + r"""Data format for the response. + + Values: + json: Responses with Content-Type of application/json + """ + json = 0 + + alt = _messages.EnumField('AltValueValuesEnum', 1, default=u'json') + fields = _messages.StringField(2) + key = _messages.StringField(3) + oauth_token = _messages.StringField(4) + prettyPrint = _messages.BooleanField(5, default=True) + quotaUser = _messages.StringField(6) + trace = _messages.StringField(7) + userIp = _messages.StringField(8) + + +class StorageBucketAccessControlsDeleteRequest(_messages.Message): + r"""A StorageBucketAccessControlsDeleteRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageBucketAccessControlsDeleteResponse(_messages.Message): + r"""An empty StorageBucketAccessControlsDelete response.""" + + +class StorageBucketAccessControlsGetRequest(_messages.Message): + r"""A StorageBucketAccessControlsGetRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageBucketAccessControlsInsertRequest(_messages.Message): + r"""A StorageBucketAccessControlsInsertRequest object. + + Fields: + bucket: Name of a bucket. + bucketAccessControl: A BucketAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + bucketAccessControl = _messages.MessageField('BucketAccessControl', 2) + userProject = _messages.StringField(3) + + +class StorageBucketAccessControlsListRequest(_messages.Message): + r"""A StorageBucketAccessControlsListRequest object. + + Fields: + bucket: Name of a bucket. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + userProject = _messages.StringField(2) + + +class StorageBucketAccessControlsPatchRequest(_messages.Message): + r"""A StorageBucketAccessControlsPatchRequest object. + + Fields: + bucket: Name of a bucket. + bucketAccessControl: A BucketAccessControl resource to be passed as the + request body. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + bucketAccessControl = _messages.MessageField('BucketAccessControl', 2) + entity = _messages.StringField(3, required=True) + userProject = _messages.StringField(4) + + +class StorageBucketAccessControlsUpdateRequest(_messages.Message): + r"""A StorageBucketAccessControlsUpdateRequest object. + + Fields: + bucket: Name of a bucket. + bucketAccessControl: A BucketAccessControl resource to be passed as the + request body. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + bucketAccessControl = _messages.MessageField('BucketAccessControl', 2) + entity = _messages.StringField(3, required=True) + userProject = _messages.StringField(4) + + +class StorageBucketsDeleteRequest(_messages.Message): + r"""A StorageBucketsDeleteRequest object. + + Fields: + bucket: Name of a bucket. + ifMetagenerationMatch: If set, only deletes the bucket if its + metageneration matches this value. + ifMetagenerationNotMatch: If set, only deletes the bucket if its + metageneration does not match this value. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + ifMetagenerationMatch = _messages.IntegerField(2) + ifMetagenerationNotMatch = _messages.IntegerField(3) + userProject = _messages.StringField(4) + + +class StorageBucketsDeleteResponse(_messages.Message): + r"""An empty StorageBucketsDelete response.""" + + +class StorageBucketsGetIamPolicyRequest(_messages.Message): + r"""A StorageBucketsGetIamPolicyRequest object. + + Fields: + bucket: Name of a bucket. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + userProject = _messages.StringField(2) + + +class StorageBucketsGetRequest(_messages.Message): + r"""A StorageBucketsGetRequest object. + + Enums: + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl. + + Fields: + bucket: Name of a bucket. + ifMetagenerationMatch: Makes the return of the bucket metadata conditional + on whether the bucket's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the return of the bucket metadata + conditional on whether the bucket's current metageneration does not + match the given value. + projection: Set of properties to return. Defaults to noAcl. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl. + + Values: + full: Include all properties. + noAcl: Omit owner, acl and defaultObjectAcl properties. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + ifMetagenerationMatch = _messages.IntegerField(2) + ifMetagenerationNotMatch = _messages.IntegerField(3) + projection = _messages.EnumField('ProjectionValueValuesEnum', 4) + userProject = _messages.StringField(5) + + +class StorageBucketsInsertRequest(_messages.Message): + r"""A StorageBucketsInsertRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this bucket. + PredefinedDefaultObjectAclValueValuesEnum: Apply a predefined set of + default object access controls to this bucket. + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl, + unless the bucket resource specifies acl or defaultObjectAcl properties, + when it defaults to full. + + Fields: + bucket: A Bucket resource to be passed as the request body. + predefinedAcl: Apply a predefined set of access controls to this bucket. + predefinedDefaultObjectAcl: Apply a predefined set of default object + access controls to this bucket. + project: A valid API project identifier. + projection: Set of properties to return. Defaults to noAcl, unless the + bucket resource specifies acl or defaultObjectAcl properties, when it + defaults to full. + userProject: The project to be billed for this request. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this bucket. + + Values: + authenticatedRead: Project team owners get OWNER access, and + allAuthenticatedUsers get READER access. + private: Project team owners get OWNER access. + projectPrivate: Project team members get access according to their + roles. + publicRead: Project team owners get OWNER access, and allUsers get + READER access. + publicReadWrite: Project team owners get OWNER access, and allUsers get + WRITER access. + """ + authenticatedRead = 0 + private = 1 + projectPrivate = 2 + publicRead = 3 + publicReadWrite = 4 + + class PredefinedDefaultObjectAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of default object access controls to this + bucket. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl, unless the bucket + resource specifies acl or defaultObjectAcl properties, when it defaults to + full. + + Values: + full: Include all properties. + noAcl: Omit owner, acl and defaultObjectAcl properties. + """ + full = 0 + noAcl = 1 + + bucket = _messages.MessageField('Bucket', 1) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 2) + predefinedDefaultObjectAcl = _messages.EnumField( + 'PredefinedDefaultObjectAclValueValuesEnum', 3) + project = _messages.StringField(4, required=True) + projection = _messages.EnumField('ProjectionValueValuesEnum', 5) + userProject = _messages.StringField(6) + + +class StorageBucketsListRequest(_messages.Message): + r"""A StorageBucketsListRequest object. + + Enums: + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl. + + Fields: + maxResults: Maximum number of buckets to return in a single response. The + service will use this parameter or 1,000 items, whichever is smaller. + pageToken: A previously-returned page token representing part of the + larger set of results to view. + prefix: Filter results to buckets whose names begin with this prefix. + project: A valid API project identifier. + projection: Set of properties to return. Defaults to noAcl. + userProject: The project to be billed for this request. + """ + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl. + + Values: + full: Include all properties. + noAcl: Omit owner, acl and defaultObjectAcl properties. + """ + full = 0 + noAcl = 1 + + maxResults = _messages.IntegerField( + 1, variant=_messages.Variant.UINT32, default=1000) + pageToken = _messages.StringField(2) + prefix = _messages.StringField(3) + project = _messages.StringField(4, required=True) + projection = _messages.EnumField('ProjectionValueValuesEnum', 5) + userProject = _messages.StringField(6) + + +class StorageBucketsLockRetentionPolicyRequest(_messages.Message): + r"""A StorageBucketsLockRetentionPolicyRequest object. + + Fields: + bucket: Name of a bucket. + ifMetagenerationMatch: Makes the operation conditional on whether bucket's + current metageneration matches the given value. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + ifMetagenerationMatch = _messages.IntegerField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageBucketsPatchRequest(_messages.Message): + r"""A StorageBucketsPatchRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this bucket. + PredefinedDefaultObjectAclValueValuesEnum: Apply a predefined set of + default object access controls to this bucket. + ProjectionValueValuesEnum: Set of properties to return. Defaults to full. + + Fields: + bucket: Name of a bucket. + bucketResource: A Bucket resource to be passed as the request body. + ifMetagenerationMatch: Makes the return of the bucket metadata conditional + on whether the bucket's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the return of the bucket metadata + conditional on whether the bucket's current metageneration does not + match the given value. + predefinedAcl: Apply a predefined set of access controls to this bucket. + predefinedDefaultObjectAcl: Apply a predefined set of default object + access controls to this bucket. + projection: Set of properties to return. Defaults to full. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this bucket. + + Values: + authenticatedRead: Project team owners get OWNER access, and + allAuthenticatedUsers get READER access. + private: Project team owners get OWNER access. + projectPrivate: Project team members get access according to their + roles. + publicRead: Project team owners get OWNER access, and allUsers get + READER access. + publicReadWrite: Project team owners get OWNER access, and allUsers get + WRITER access. + """ + authenticatedRead = 0 + private = 1 + projectPrivate = 2 + publicRead = 3 + publicReadWrite = 4 + + class PredefinedDefaultObjectAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of default object access controls to this + bucket. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to full. + + Values: + full: Include all properties. + noAcl: Omit owner, acl and defaultObjectAcl properties. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + bucketResource = _messages.MessageField('Bucket', 2) + ifMetagenerationMatch = _messages.IntegerField(3) + ifMetagenerationNotMatch = _messages.IntegerField(4) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 5) + predefinedDefaultObjectAcl = _messages.EnumField( + 'PredefinedDefaultObjectAclValueValuesEnum', 6) + projection = _messages.EnumField('ProjectionValueValuesEnum', 7) + userProject = _messages.StringField(8) + + +class StorageBucketsSetIamPolicyRequest(_messages.Message): + r"""A StorageBucketsSetIamPolicyRequest object. + + Fields: + bucket: Name of a bucket. + policy: A Policy resource to be passed as the request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + policy = _messages.MessageField('Policy', 2) + userProject = _messages.StringField(3) + + +class StorageBucketsTestIamPermissionsRequest(_messages.Message): + r"""A StorageBucketsTestIamPermissionsRequest object. + + Fields: + bucket: Name of a bucket. + permissions: Permissions to test. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + permissions = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageBucketsUpdateRequest(_messages.Message): + r"""A StorageBucketsUpdateRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this bucket. + PredefinedDefaultObjectAclValueValuesEnum: Apply a predefined set of + default object access controls to this bucket. + ProjectionValueValuesEnum: Set of properties to return. Defaults to full. + + Fields: + bucket: Name of a bucket. + bucketResource: A Bucket resource to be passed as the request body. + ifMetagenerationMatch: Makes the return of the bucket metadata conditional + on whether the bucket's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the return of the bucket metadata + conditional on whether the bucket's current metageneration does not + match the given value. + predefinedAcl: Apply a predefined set of access controls to this bucket. + predefinedDefaultObjectAcl: Apply a predefined set of default object + access controls to this bucket. + projection: Set of properties to return. Defaults to full. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this bucket. + + Values: + authenticatedRead: Project team owners get OWNER access, and + allAuthenticatedUsers get READER access. + private: Project team owners get OWNER access. + projectPrivate: Project team members get access according to their + roles. + publicRead: Project team owners get OWNER access, and allUsers get + READER access. + publicReadWrite: Project team owners get OWNER access, and allUsers get + WRITER access. + """ + authenticatedRead = 0 + private = 1 + projectPrivate = 2 + publicRead = 3 + publicReadWrite = 4 + + class PredefinedDefaultObjectAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of default object access controls to this + bucket. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to full. + + Values: + full: Include all properties. + noAcl: Omit owner, acl and defaultObjectAcl properties. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + bucketResource = _messages.MessageField('Bucket', 2) + ifMetagenerationMatch = _messages.IntegerField(3) + ifMetagenerationNotMatch = _messages.IntegerField(4) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 5) + predefinedDefaultObjectAcl = _messages.EnumField( + 'PredefinedDefaultObjectAclValueValuesEnum', 6) + projection = _messages.EnumField('ProjectionValueValuesEnum', 7) + userProject = _messages.StringField(8) + + +class StorageChannelsStopResponse(_messages.Message): + r"""An empty StorageChannelsStop response.""" + + +class StorageDefaultObjectAccessControlsDeleteRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsDeleteRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageDefaultObjectAccessControlsDeleteResponse(_messages.Message): + r"""An empty StorageDefaultObjectAccessControlsDelete response.""" + + +class StorageDefaultObjectAccessControlsGetRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsGetRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageDefaultObjectAccessControlsInsertRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsInsertRequest object. + + Fields: + bucket: Name of a bucket. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 2) + userProject = _messages.StringField(3) + + +class StorageDefaultObjectAccessControlsListRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsListRequest object. + + Fields: + bucket: Name of a bucket. + ifMetagenerationMatch: If present, only return default ACL listing if the + bucket's current metageneration matches this value. + ifMetagenerationNotMatch: If present, only return default ACL listing if + the bucket's current metageneration does not match the given value. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + ifMetagenerationMatch = _messages.IntegerField(2) + ifMetagenerationNotMatch = _messages.IntegerField(3) + userProject = _messages.StringField(4) + + +class StorageDefaultObjectAccessControlsPatchRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsPatchRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 3) + userProject = _messages.StringField(4) + + +class StorageDefaultObjectAccessControlsUpdateRequest(_messages.Message): + r"""A StorageDefaultObjectAccessControlsUpdateRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 3) + userProject = _messages.StringField(4) + + +class StorageNotificationsDeleteRequest(_messages.Message): + r"""A StorageNotificationsDeleteRequest object. + + Fields: + bucket: The parent bucket of the notification. + notification: ID of the notification to delete. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + notification = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageNotificationsDeleteResponse(_messages.Message): + r"""An empty StorageNotificationsDelete response.""" + + +class StorageNotificationsGetRequest(_messages.Message): + r"""A StorageNotificationsGetRequest object. + + Fields: + bucket: The parent bucket of the notification. + notification: Notification ID + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + notification = _messages.StringField(2, required=True) + userProject = _messages.StringField(3) + + +class StorageNotificationsInsertRequest(_messages.Message): + r"""A StorageNotificationsInsertRequest object. + + Fields: + bucket: The parent bucket of the notification. + notification: A Notification resource to be passed as the request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + notification = _messages.MessageField('Notification', 2) + userProject = _messages.StringField(3) + + +class StorageNotificationsListRequest(_messages.Message): + r"""A StorageNotificationsListRequest object. + + Fields: + bucket: Name of a Google Cloud Storage bucket. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + userProject = _messages.StringField(2) + + +class StorageObjectAccessControlsDeleteRequest(_messages.Message): + r"""A StorageObjectAccessControlsDeleteRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + generation = _messages.IntegerField(3) + object = _messages.StringField(4, required=True) + userProject = _messages.StringField(5) + + +class StorageObjectAccessControlsDeleteResponse(_messages.Message): + r"""An empty StorageObjectAccessControlsDelete response.""" + + +class StorageObjectAccessControlsGetRequest(_messages.Message): + r"""A StorageObjectAccessControlsGetRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + generation = _messages.IntegerField(3) + object = _messages.StringField(4, required=True) + userProject = _messages.StringField(5) + + +class StorageObjectAccessControlsInsertRequest(_messages.Message): + r"""A StorageObjectAccessControlsInsertRequest object. + + Fields: + bucket: Name of a bucket. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + object = _messages.StringField(3, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 4) + userProject = _messages.StringField(5) + + +class StorageObjectAccessControlsListRequest(_messages.Message): + r"""A StorageObjectAccessControlsListRequest object. + + Fields: + bucket: Name of a bucket. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + object = _messages.StringField(3, required=True) + userProject = _messages.StringField(4) + + +class StorageObjectAccessControlsPatchRequest(_messages.Message): + r"""A StorageObjectAccessControlsPatchRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + generation = _messages.IntegerField(3) + object = _messages.StringField(4, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 5) + userProject = _messages.StringField(6) + + +class StorageObjectAccessControlsUpdateRequest(_messages.Message): + r"""A StorageObjectAccessControlsUpdateRequest object. + + Fields: + bucket: Name of a bucket. + entity: The entity holding the permission. Can be user-userId, user- + emailAddress, group-groupId, group-emailAddress, allUsers, or + allAuthenticatedUsers. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + objectAccessControl: A ObjectAccessControl resource to be passed as the + request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + entity = _messages.StringField(2, required=True) + generation = _messages.IntegerField(3) + object = _messages.StringField(4, required=True) + objectAccessControl = _messages.MessageField('ObjectAccessControl', 5) + userProject = _messages.StringField(6) + + +class StorageObjectsComposeRequest(_messages.Message): + r"""A StorageObjectsComposeRequest object. + + Enums: + DestinationPredefinedAclValueValuesEnum: Apply a predefined set of access + controls to the destination object. + + Fields: + composeRequest: A ComposeRequest resource to be passed as the request + body. + destinationBucket: Name of the bucket containing the source objects. The + destination object is stored in this bucket. + destinationObject: Name of the new object. For information about how to + URL encode object names to be path safe, see Encoding URI Path Parts. + destinationPredefinedAcl: Apply a predefined set of access controls to the + destination object. + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + kmsKeyName: Resource name of the Cloud KMS key, of the form projects/my- + project/locations/global/keyRings/my-kr/cryptoKeys/my-key, that will be + used to encrypt the object. Overrides the object metadata's kms_key_name + value, if any. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class DestinationPredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to the destination object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + composeRequest = _messages.MessageField('ComposeRequest', 1) + destinationBucket = _messages.StringField(2, required=True) + destinationObject = _messages.StringField(3, required=True) + destinationPredefinedAcl = _messages.EnumField( + 'DestinationPredefinedAclValueValuesEnum', 4) + ifGenerationMatch = _messages.IntegerField(5) + ifMetagenerationMatch = _messages.IntegerField(6) + kmsKeyName = _messages.StringField(7) + userProject = _messages.StringField(8) + + +class StorageObjectsCopyRequest(_messages.Message): + r"""A StorageObjectsCopyRequest object. + + Enums: + DestinationPredefinedAclValueValuesEnum: Apply a predefined set of access + controls to the destination object. + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl, + unless the object resource specifies the acl property, when it defaults + to full. + + Fields: + destinationBucket: Name of the bucket in which to store the new object. + Overrides the provided object metadata's bucket value, if any.For + information about how to URL encode object names to be path safe, see + Encoding URI Path Parts. + destinationObject: Name of the new object. Required when the object + metadata is not otherwise provided. Overrides the object metadata's name + value, if any. + destinationPredefinedAcl: Apply a predefined set of access controls to the + destination object. + ifGenerationMatch: Makes the operation conditional on whether the + destination object's current generation matches the given value. Setting + to 0 makes the operation succeed only if there are no live versions of + the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + destination object's current generation does not match the given value. + If no live object exists, the precondition fails. Setting to 0 makes the + operation succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + destination object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + destination object's current metageneration does not match the given + value. + ifSourceGenerationMatch: Makes the operation conditional on whether the + source object's current generation matches the given value. + ifSourceGenerationNotMatch: Makes the operation conditional on whether the + source object's current generation does not match the given value. + ifSourceMetagenerationMatch: Makes the operation conditional on whether + the source object's current metageneration matches the given value. + ifSourceMetagenerationNotMatch: Makes the operation conditional on whether + the source object's current metageneration does not match the given + value. + object: A Object resource to be passed as the request body. + projection: Set of properties to return. Defaults to noAcl, unless the + object resource specifies the acl property, when it defaults to full. + sourceBucket: Name of the bucket in which to find the source object. + sourceGeneration: If present, selects a specific revision of the source + object (as opposed to the latest version, the default). + sourceObject: Name of the source object. For information about how to URL + encode object names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class DestinationPredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to the destination object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl, unless the object + resource specifies the acl property, when it defaults to full. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + destinationBucket = _messages.StringField(1, required=True) + destinationObject = _messages.StringField(2, required=True) + destinationPredefinedAcl = _messages.EnumField( + 'DestinationPredefinedAclValueValuesEnum', 3) + ifGenerationMatch = _messages.IntegerField(4) + ifGenerationNotMatch = _messages.IntegerField(5) + ifMetagenerationMatch = _messages.IntegerField(6) + ifMetagenerationNotMatch = _messages.IntegerField(7) + ifSourceGenerationMatch = _messages.IntegerField(8) + ifSourceGenerationNotMatch = _messages.IntegerField(9) + ifSourceMetagenerationMatch = _messages.IntegerField(10) + ifSourceMetagenerationNotMatch = _messages.IntegerField(11) + object = _messages.MessageField('Object', 12) + projection = _messages.EnumField('ProjectionValueValuesEnum', 13) + sourceBucket = _messages.StringField(14, required=True) + sourceGeneration = _messages.IntegerField(15) + sourceObject = _messages.StringField(16, required=True) + userProject = _messages.StringField(17) + + +class StorageObjectsDeleteRequest(_messages.Message): + r"""A StorageObjectsDeleteRequest object. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, permanently deletes a specific revision of this + object (as opposed to the latest version, the default). + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + object's current metageneration does not match the given value. + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + ifGenerationMatch = _messages.IntegerField(3) + ifGenerationNotMatch = _messages.IntegerField(4) + ifMetagenerationMatch = _messages.IntegerField(5) + ifMetagenerationNotMatch = _messages.IntegerField(6) + object = _messages.StringField(7, required=True) + userProject = _messages.StringField(8) + + +class StorageObjectsDeleteResponse(_messages.Message): + r"""An empty StorageObjectsDelete response.""" + + +class StorageObjectsGetIamPolicyRequest(_messages.Message): + r"""A StorageObjectsGetIamPolicyRequest object. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + object = _messages.StringField(3, required=True) + userProject = _messages.StringField(4) + + +class StorageObjectsGetRequest(_messages.Message): + r"""A StorageObjectsGetRequest object. + + Enums: + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + object's current metageneration does not match the given value. + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + projection: Set of properties to return. Defaults to noAcl. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + ifGenerationMatch = _messages.IntegerField(3) + ifGenerationNotMatch = _messages.IntegerField(4) + ifMetagenerationMatch = _messages.IntegerField(5) + ifMetagenerationNotMatch = _messages.IntegerField(6) + object = _messages.StringField(7, required=True) + projection = _messages.EnumField('ProjectionValueValuesEnum', 8) + userProject = _messages.StringField(9) + + +class StorageObjectsInsertRequest(_messages.Message): + r"""A StorageObjectsInsertRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this object. + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl, + unless the object resource specifies the acl property, when it defaults + to full. + + Fields: + bucket: Name of the bucket in which to store the new object. Overrides the + provided object metadata's bucket value, if any. + contentEncoding: If set, sets the contentEncoding property of the final + object to this value. Setting this parameter is equivalent to setting + the contentEncoding metadata property. This can be useful when uploading + an object with uploadType=media to indicate the encoding of the content + being uploaded. + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + object's current metageneration does not match the given value. + kmsKeyName: Resource name of the Cloud KMS key, of the form projects/my- + project/locations/global/keyRings/my-kr/cryptoKeys/my-key, that will be + used to encrypt the object. Overrides the object metadata's kms_key_name + value, if any. + name: Name of the object. Required when the object metadata is not + otherwise provided. Overrides the object metadata's name value, if any. + For information about how to URL encode object names to be path safe, + see Encoding URI Path Parts. + object: A Object resource to be passed as the request body. + predefinedAcl: Apply a predefined set of access controls to this object. + projection: Set of properties to return. Defaults to noAcl, unless the + object resource specifies the acl property, when it defaults to full. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl, unless the object + resource specifies the acl property, when it defaults to full. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + contentEncoding = _messages.StringField(2) + ifGenerationMatch = _messages.IntegerField(3) + ifGenerationNotMatch = _messages.IntegerField(4) + ifMetagenerationMatch = _messages.IntegerField(5) + ifMetagenerationNotMatch = _messages.IntegerField(6) + kmsKeyName = _messages.StringField(7) + name = _messages.StringField(8) + object = _messages.MessageField('Object', 9) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 10) + projection = _messages.EnumField('ProjectionValueValuesEnum', 11) + userProject = _messages.StringField(12) + + +class StorageObjectsListRequest(_messages.Message): + r"""A StorageObjectsListRequest object. + + Enums: + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl. + + Fields: + bucket: Name of the bucket in which to look for objects. + delimiter: Returns results in a directory-like mode. items will contain + only objects whose names, aside from the prefix, do not contain + delimiter. Objects whose names, aside from the prefix, contain delimiter + will have their name, truncated after the delimiter, returned in + prefixes. Duplicate prefixes are omitted. + includeTrailingDelimiter: If true, objects that end in exactly one + instance of delimiter will have their metadata included in items in + addition to prefixes. + maxResults: Maximum number of items plus prefixes to return in a single + page of responses. As duplicate prefixes are omitted, fewer total + results may be returned than requested. The service will use this + parameter or 1,000 items, whichever is smaller. + pageToken: A previously-returned page token representing part of the + larger set of results to view. + prefix: Filter results to objects whose names begin with this prefix. + projection: Set of properties to return. Defaults to noAcl. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + versions: If true, lists all versions of an object as distinct results. + The default is false. For more information, see Object Versioning. + """ + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + delimiter = _messages.StringField(2) + includeTrailingDelimiter = _messages.BooleanField(3) + maxResults = _messages.IntegerField( + 4, variant=_messages.Variant.UINT32, default=1000) + pageToken = _messages.StringField(5) + prefix = _messages.StringField(6) + projection = _messages.EnumField('ProjectionValueValuesEnum', 7) + userProject = _messages.StringField(8) + versions = _messages.BooleanField(9) + + +class StorageObjectsPatchRequest(_messages.Message): + r"""A StorageObjectsPatchRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this object. + ProjectionValueValuesEnum: Set of properties to return. Defaults to full. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + object's current metageneration does not match the given value. + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + objectResource: A Object resource to be passed as the request body. + predefinedAcl: Apply a predefined set of access controls to this object. + projection: Set of properties to return. Defaults to full. + userProject: The project to be billed for this request, for Requester Pays + buckets. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to full. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + ifGenerationMatch = _messages.IntegerField(3) + ifGenerationNotMatch = _messages.IntegerField(4) + ifMetagenerationMatch = _messages.IntegerField(5) + ifMetagenerationNotMatch = _messages.IntegerField(6) + object = _messages.StringField(7, required=True) + objectResource = _messages.MessageField('Object', 8) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 9) + projection = _messages.EnumField('ProjectionValueValuesEnum', 10) + userProject = _messages.StringField(11) + + +class StorageObjectsRewriteRequest(_messages.Message): + r"""A StorageObjectsRewriteRequest object. + + Enums: + DestinationPredefinedAclValueValuesEnum: Apply a predefined set of access + controls to the destination object. + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl, + unless the object resource specifies the acl property, when it defaults + to full. + + Fields: + destinationBucket: Name of the bucket in which to store the new object. + Overrides the provided object metadata's bucket value, if any. + destinationKmsKeyName: Resource name of the Cloud KMS key, of the form + projects/my-project/locations/global/keyRings/my-kr/cryptoKeys/my-key, + that will be used to encrypt the object. Overrides the object metadata's + kms_key_name value, if any. + destinationObject: Name of the new object. Required when the object + metadata is not otherwise provided. Overrides the object metadata's name + value, if any. For information about how to URL encode object names to + be path safe, see Encoding URI Path Parts. + destinationPredefinedAcl: Apply a predefined set of access controls to the + destination object. + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + destination object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + destination object's current metageneration does not match the given + value. + ifSourceGenerationMatch: Makes the operation conditional on whether the + source object's current generation matches the given value. + ifSourceGenerationNotMatch: Makes the operation conditional on whether the + source object's current generation does not match the given value. + ifSourceMetagenerationMatch: Makes the operation conditional on whether + the source object's current metageneration matches the given value. + ifSourceMetagenerationNotMatch: Makes the operation conditional on whether + the source object's current metageneration does not match the given + value. + maxBytesRewrittenPerCall: The maximum number of bytes that will be + rewritten per rewrite request. Most callers shouldn't need to specify + this parameter - it is primarily in place to support testing. If + specified the value must be an integral multiple of 1 MiB (1048576). + Also, this only applies to requests where the source and destination + span locations and/or storage classes. Finally, this value must not + change across rewrite calls else you'll get an error that the + rewriteToken is invalid. + object: A Object resource to be passed as the request body. + projection: Set of properties to return. Defaults to noAcl, unless the + object resource specifies the acl property, when it defaults to full. + rewriteToken: Include this field (from the previous rewrite response) on + each rewrite request after the first one, until the rewrite response + 'done' flag is true. Calls that provide a rewriteToken can omit all + other request fields, but if included those fields must match the values + provided in the first rewrite request. + sourceBucket: Name of the bucket in which to find the source object. + sourceGeneration: If present, selects a specific revision of the source + object (as opposed to the latest version, the default). + sourceObject: Name of the source object. For information about how to URL + encode object names to be path safe, see Encoding URI Path Parts. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class DestinationPredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to the destination object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl, unless the object + resource specifies the acl property, when it defaults to full. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + destinationBucket = _messages.StringField(1, required=True) + destinationKmsKeyName = _messages.StringField(2) + destinationObject = _messages.StringField(3, required=True) + destinationPredefinedAcl = _messages.EnumField( + 'DestinationPredefinedAclValueValuesEnum', 4) + ifGenerationMatch = _messages.IntegerField(5) + ifGenerationNotMatch = _messages.IntegerField(6) + ifMetagenerationMatch = _messages.IntegerField(7) + ifMetagenerationNotMatch = _messages.IntegerField(8) + ifSourceGenerationMatch = _messages.IntegerField(9) + ifSourceGenerationNotMatch = _messages.IntegerField(10) + ifSourceMetagenerationMatch = _messages.IntegerField(11) + ifSourceMetagenerationNotMatch = _messages.IntegerField(12) + maxBytesRewrittenPerCall = _messages.IntegerField(13) + object = _messages.MessageField('Object', 14) + projection = _messages.EnumField('ProjectionValueValuesEnum', 15) + rewriteToken = _messages.StringField(16) + sourceBucket = _messages.StringField(17, required=True) + sourceGeneration = _messages.IntegerField(18) + sourceObject = _messages.StringField(19, required=True) + userProject = _messages.StringField(20) + + +class StorageObjectsSetIamPolicyRequest(_messages.Message): + r"""A StorageObjectsSetIamPolicyRequest object. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + policy: A Policy resource to be passed as the request body. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + object = _messages.StringField(3, required=True) + policy = _messages.MessageField('Policy', 4) + userProject = _messages.StringField(5) + + +class StorageObjectsTestIamPermissionsRequest(_messages.Message): + r"""A StorageObjectsTestIamPermissionsRequest object. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + permissions: Permissions to test. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + object = _messages.StringField(3, required=True) + permissions = _messages.StringField(4, required=True) + userProject = _messages.StringField(5) + + +class StorageObjectsUpdateRequest(_messages.Message): + r"""A StorageObjectsUpdateRequest object. + + Enums: + PredefinedAclValueValuesEnum: Apply a predefined set of access controls to + this object. + ProjectionValueValuesEnum: Set of properties to return. Defaults to full. + + Fields: + bucket: Name of the bucket in which the object resides. + generation: If present, selects a specific revision of this object (as + opposed to the latest version, the default). + ifGenerationMatch: Makes the operation conditional on whether the object's + current generation matches the given value. Setting to 0 makes the + operation succeed only if there are no live versions of the object. + ifGenerationNotMatch: Makes the operation conditional on whether the + object's current generation does not match the given value. If no live + object exists, the precondition fails. Setting to 0 makes the operation + succeed only if there is a live version of the object. + ifMetagenerationMatch: Makes the operation conditional on whether the + object's current metageneration matches the given value. + ifMetagenerationNotMatch: Makes the operation conditional on whether the + object's current metageneration does not match the given value. + object: Name of the object. For information about how to URL encode object + names to be path safe, see Encoding URI Path Parts. + objectResource: A Object resource to be passed as the request body. + predefinedAcl: Apply a predefined set of access controls to this object. + projection: Set of properties to return. Defaults to full. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + """ + class PredefinedAclValueValuesEnum(_messages.Enum): + r"""Apply a predefined set of access controls to this object. + + Values: + authenticatedRead: Object owner gets OWNER access, and + allAuthenticatedUsers get READER access. + bucketOwnerFullControl: Object owner gets OWNER access, and project team + owners get OWNER access. + bucketOwnerRead: Object owner gets OWNER access, and project team owners + get READER access. + private: Object owner gets OWNER access. + projectPrivate: Object owner gets OWNER access, and project team members + get access according to their roles. + publicRead: Object owner gets OWNER access, and allUsers get READER + access. + """ + authenticatedRead = 0 + bucketOwnerFullControl = 1 + bucketOwnerRead = 2 + private = 3 + projectPrivate = 4 + publicRead = 5 + + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to full. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + generation = _messages.IntegerField(2) + ifGenerationMatch = _messages.IntegerField(3) + ifGenerationNotMatch = _messages.IntegerField(4) + ifMetagenerationMatch = _messages.IntegerField(5) + ifMetagenerationNotMatch = _messages.IntegerField(6) + object = _messages.StringField(7, required=True) + objectResource = _messages.MessageField('Object', 8) + predefinedAcl = _messages.EnumField('PredefinedAclValueValuesEnum', 9) + projection = _messages.EnumField('ProjectionValueValuesEnum', 10) + userProject = _messages.StringField(11) + + +class StorageObjectsWatchAllRequest(_messages.Message): + r"""A StorageObjectsWatchAllRequest object. + + Enums: + ProjectionValueValuesEnum: Set of properties to return. Defaults to noAcl. + + Fields: + bucket: Name of the bucket in which to look for objects. + channel: A Channel resource to be passed as the request body. + delimiter: Returns results in a directory-like mode. items will contain + only objects whose names, aside from the prefix, do not contain + delimiter. Objects whose names, aside from the prefix, contain delimiter + will have their name, truncated after the delimiter, returned in + prefixes. Duplicate prefixes are omitted. + includeTrailingDelimiter: If true, objects that end in exactly one + instance of delimiter will have their metadata included in items in + addition to prefixes. + maxResults: Maximum number of items plus prefixes to return in a single + page of responses. As duplicate prefixes are omitted, fewer total + results may be returned than requested. The service will use this + parameter or 1,000 items, whichever is smaller. + pageToken: A previously-returned page token representing part of the + larger set of results to view. + prefix: Filter results to objects whose names begin with this prefix. + projection: Set of properties to return. Defaults to noAcl. + userProject: The project to be billed for this request. Required for + Requester Pays buckets. + versions: If true, lists all versions of an object as distinct results. + The default is false. For more information, see Object Versioning. + """ + class ProjectionValueValuesEnum(_messages.Enum): + r"""Set of properties to return. Defaults to noAcl. + + Values: + full: Include all properties. + noAcl: Omit the owner, acl property. + """ + full = 0 + noAcl = 1 + + bucket = _messages.StringField(1, required=True) + channel = _messages.MessageField('Channel', 2) + delimiter = _messages.StringField(3) + includeTrailingDelimiter = _messages.BooleanField(4) + maxResults = _messages.IntegerField( + 5, variant=_messages.Variant.UINT32, default=1000) + pageToken = _messages.StringField(6) + prefix = _messages.StringField(7) + projection = _messages.EnumField('ProjectionValueValuesEnum', 8) + userProject = _messages.StringField(9) + versions = _messages.BooleanField(10) + + +class StorageProjectsServiceAccountGetRequest(_messages.Message): + r"""A StorageProjectsServiceAccountGetRequest object. + + Fields: + projectId: Project ID + userProject: The project to be billed for this request. + """ + + projectId = _messages.StringField(1, required=True) + userProject = _messages.StringField(2) + + +class TestIamPermissionsResponse(_messages.Message): + r"""A storage.(buckets|objects).testIamPermissions response. + + Fields: + kind: The kind of item this is. + permissions: The permissions held by the caller. Permissions are always of + the format storage.resource.capability, where resource is one of buckets + or objects. The supported permissions are as follows: - + storage.buckets.delete - Delete bucket. - storage.buckets.get - Read + bucket metadata. - storage.buckets.getIamPolicy - Read bucket IAM + policy. - storage.buckets.create - Create bucket. - + storage.buckets.list - List buckets. - storage.buckets.setIamPolicy - + Update bucket IAM policy. - storage.buckets.update - Update bucket + metadata. - storage.objects.delete - Delete object. - + storage.objects.get - Read object data and metadata. - + storage.objects.getIamPolicy - Read object IAM policy. - + storage.objects.create - Create object. - storage.objects.list - List + objects. - storage.objects.setIamPolicy - Update object IAM policy. + - storage.objects.update - Update object metadata. + """ + + kind = _messages.StringField(1, default=u'storage#testIamPermissionsResponse') + permissions = _messages.StringField(2, repeated=True) diff --git a/sdks/python/apache_beam/options/pipeline_options_validator_test.py b/sdks/python/apache_beam/options/pipeline_options_validator_test.py index 56f305a01b74..015e3ef787f8 100644 --- a/sdks/python/apache_beam/options/pipeline_options_validator_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_validator_test.py @@ -106,7 +106,6 @@ def test_missing_required_options(self): ]), ('gs://foo', 'gs://foo/bar', []), ('gs://foo/', 'gs://foo/bar', []), ('gs://foo/bar', 'gs://foo/bar', []) ]) - @unittest.skip('Not compatible with new GCS client. See GH issue #26335.') def test_gcs_path(self, temp_location, staging_location, expected_error_args): def get_validator(_temp_location, _staging_location): options = ['--project=example:example', '--job_name=job'] diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index d2ba457df529..bbcf880efdf7 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -55,6 +55,7 @@ from apache_beam.internal.http_client import get_new_http from apache_beam.io.filesystems import FileSystems from apache_beam.io.gcp.gcsfilesystem import GCSFileSystem +from apache_beam.io.gcp.internal.clients import storage from apache_beam.options.pipeline_options import DebugOptions from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import StandardOptions @@ -490,16 +491,13 @@ def __init__(self, options, root_staging_location=None): self._enable_caching = self.google_cloud_options.enable_artifact_caching self._root_staging_location = ( root_staging_location or self.google_cloud_options.staging_location) - self.environment_version = _FNAPI_ENVIRONMENT_MAJOR_VERSION - from google.cloud import storage + self.environment_version = _FNAPI_ENVIRONMENT_MAJOR_VERSION if self.google_cloud_options.no_auth: credentials = None - storage_credentials = None else: credentials = get_service_credentials(options) - storage_credentials = credentials.get_google_auth_credentials() http_client = get_new_http() self._client = dataflow.DataflowV1b3( @@ -508,10 +506,12 @@ def __init__(self, options, root_staging_location=None): get_credentials=(not self.google_cloud_options.no_auth), http=http_client, response_encoding=get_response_encoding()) - if storage_credentials: - self._storage_client = storage.Client(credentials=storage_credentials) - else: - self._storage_client = storage.Client.create_anonymous_client() + self._storage_client = storage.StorageV1( + url='https://www.googleapis.com/storage/v1', + credentials=credentials, + get_credentials=(not self.google_cloud_options.no_auth), + http=http_client, + response_encoding=get_response_encoding()) self._sdk_image_overrides = self._get_sdk_image_overrides(options) def _get_sdk_image_overrides(self, pipeline_options): @@ -654,8 +654,6 @@ def stage_file( mime_type='application/octet-stream', total_size=None): """Stages a file at a GCS or local path with stream-supplied contents.""" - from google.cloud.exceptions import Forbidden - from google.cloud.exceptions import NotFound if not gcs_or_local_path.startswith('gs://'): local_path = FileSystems.join(gcs_or_local_path, file_name) _LOGGER.info('Staging file locally to %s', local_path) @@ -663,35 +661,31 @@ def stage_file( f.write(stream.read()) return gcs_location = FileSystems.join(gcs_or_local_path, file_name) - bucket_name, blob_name = gcs_location[5:].split('/', 1) + bucket, name = gcs_location[5:].split('/', 1) + + request = storage.StorageObjectsInsertRequest(bucket=bucket, name=name) start_time = time.time() _LOGGER.info('Starting GCS upload to %s...', gcs_location) + upload = storage.Upload(stream, mime_type, total_size) try: - from google.cloud.storage import Blob - from google.cloud.storage.fileio import BlobWriter - bucket = self._storage_client.get_bucket(bucket_name) - blob = bucket.get_blob(blob_name) - if not blob: - blob = Blob(blob_name, bucket) - with BlobWriter(blob) as f: - f.write(stream.read()) - _LOGGER.info( - 'Completed GCS upload to %s in %s seconds.', - gcs_location, - int(time.time() - start_time)) - return - except Exception as e: - reportable_errors = [ - Forbidden, - NotFound, - ] - if type(e) in reportable_errors: + response = self._storage_client.objects.Insert(request, upload=upload) + except exceptions.HttpError as e: + reportable_errors = { + 403: 'access denied', + 404: 'bucket not found', + } + if e.status_code in reportable_errors: raise IOError(( 'Could not upload to GCS path %s: %s. Please verify ' - 'that credentials are valid, that the specified path ' - 'exists, and that you have write access to it.') % - (gcs_or_local_path, e)) + 'that credentials are valid and that you have write ' + 'access to the specified path.') % + (gcs_or_local_path, reportable_errors[e.status_code])) raise + _LOGGER.info( + 'Completed GCS upload to %s in %s seconds.', + gcs_location, + int(time.time() - start_time)) + return response @retry.no_retries # Using no_retries marks this as an integration point. def create_job(self, job): diff --git a/sdks/python/apache_beam/runners/interactive/utils.py b/sdks/python/apache_beam/runners/interactive/utils.py index b5fcee48c293..bdd9ab4d1a89 100644 --- a/sdks/python/apache_beam/runners/interactive/utils.py +++ b/sdks/python/apache_beam/runners/interactive/utils.py @@ -33,6 +33,8 @@ from apache_beam.dataframe.convert import to_pcollection from apache_beam.dataframe.frame_base import DeferredBase from apache_beam.internal.gcp import auth +from apache_beam.internal.http_client import get_new_http +from apache_beam.io.gcp.internal.clients import storage from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import Pipeline from apache_beam.portability.api import beam_runner_api_pb2 @@ -449,23 +451,21 @@ def assert_bucket_exists(bucket_name): Logs a warning if the bucket cannot be verified to exist. """ try: - from google.cloud.exceptions import ClientError - from google.cloud.exceptions import NotFound - from google.cloud import storage - credentials = auth.get_service_credentials(PipelineOptions()) - if credentials: - storage_client = storage.Client(credentials=credentials) - else: - storage_client = storage.Client.create_anonymous_client() - storage_client.get_bucket(bucket_name) - except ClientError as e: - if isinstance(e, NotFound): + from apitools.base.py.exceptions import HttpError + storage_client = storage.StorageV1( + credentials=auth.get_service_credentials(PipelineOptions()), + get_credentials=False, + http=get_new_http(), + response_encoding='utf8') + request = storage.StorageBucketsGetRequest(bucket=bucket_name) + storage_client.buckets.Get(request) + except HttpError as e: + if e.status_code == 404: _LOGGER.error('%s bucket does not exist!', bucket_name) raise ValueError('Invalid GCS bucket provided!') else: _LOGGER.warning( - 'ClientError - unable to verify whether bucket %s exists', - bucket_name) + 'HttpError - unable to verify whether bucket %s exists', bucket_name) except ImportError: _LOGGER.warning( 'ImportError - unable to verify whether bucket %s exists', bucket_name) diff --git a/sdks/python/apache_beam/runners/interactive/utils_test.py b/sdks/python/apache_beam/runners/interactive/utils_test.py index f3d7f96b0dbb..ecb71a2bdef5 100644 --- a/sdks/python/apache_beam/runners/interactive/utils_test.py +++ b/sdks/python/apache_beam/runners/interactive/utils_test.py @@ -46,22 +46,27 @@ # Protect against environments where apitools library is not available. try: - from google.cloud.exceptions import BadRequest, NotFound + from apitools.base.py.exceptions import HttpError + from apitools.base.py.exceptions import HttpNotFoundError except ImportError: _http_error_imported = False + HttpError = ValueError + HttpNotFoundError = ValueError else: _http_error_imported = True -class MockStorageClient(): - def __init__(self): - pass - - def get_bucket(self, path): +class MockBuckets(): + def Get(self, path): if path == 'test-bucket-not-found': - raise NotFound('Bucket not found') + raise HttpNotFoundError({'status': 404}, {}, '') elif path == 'test-bucket-not-verified': - raise BadRequest('Request faulty') + raise HttpError({'status': 400}, {}, '') + + +class MockStorageClient(): + def __init__(self, buckets=MockBuckets()): + self.buckets = buckets class Record(NamedTuple): @@ -348,21 +353,29 @@ def test_create_var_in_main(self): self.assertIs(getattr(main_session, name, None), value) -@patch('google.cloud.storage.Client', return_value=MockStorageClient()) +@patch( + 'apache_beam.io.gcp.internal.clients.storage.StorageV1', + return_value=MockStorageClient()) @unittest.skipIf(not _http_error_imported, 'http errors are not imported.') class GCSUtilsTest(unittest.TestCase): - @patch('google.cloud.storage.Client.get_bucket') + @patch( + 'apache_beam.io.gcp.internal.clients.storage.StorageBucketsGetRequest', + return_value='test-bucket-not-found') def test_assert_bucket_exists_not_found(self, mock_response, mock_client): with self.assertRaises(ValueError): - utils.assert_bucket_exists('test-bucket-not-found') + utils.assert_bucket_exists('') - @patch('google.cloud.storage.Client.get_bucket') + @patch( + 'apache_beam.io.gcp.internal.clients.storage.StorageBucketsGetRequest', + return_value='test-bucket-not-verified') def test_assert_bucket_exists_not_verified(self, mock_response, mock_client): from apache_beam.runners.interactive.utils import _LOGGER with self.assertLogs(_LOGGER, level='WARNING'): - utils.assert_bucket_exists('test-bucket-not-verified') + utils.assert_bucket_exists('') - @patch('google.cloud.storage.Client.get_bucket') + @patch( + 'apache_beam.io.gcp.internal.clients.storage.StorageBucketsGetRequest', + return_value='test-bucket-found') def test_assert_bucket_exists_found(self, mock_response, mock_client): utils.assert_bucket_exists('') diff --git a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py index 0e8e0d97b466..19becd3e123f 100644 --- a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py +++ b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py @@ -41,6 +41,7 @@ from apache_beam import version as beam_version from apache_beam.internal.gcp.auth import get_service_credentials from apache_beam.internal.http_client import get_new_http +from apache_beam.io.gcp.internal.clients import storage from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions # pylint: disable=unused-import from apache_beam.options.pipeline_options import SetupOptions @@ -209,12 +210,12 @@ def __init__(self, options): credentials = None else: credentials = get_service_credentials(options) - from google.cloud import storage - if credentials: - self._storage_client = storage.Client( - credentials=credentials.get_google_auth_credentials()) - else: - self._storage_client = storage.Client.create_anonymous_client() + self._storage_client = storage.StorageV1( + url='https://www.googleapis.com/storage/v1', + credentials=credentials, + get_credentials=(not self._google_cloud_options.no_auth), + http=get_new_http(), + response_encoding='utf8') self._cloudbuild_client = cloudbuild.CloudbuildV1( credentials=credentials, get_credentials=(not self._google_cloud_options.no_auth), @@ -306,23 +307,27 @@ def _invoke_docker_build_and_push(self, container_image_name): "Python SDK container built and pushed as %s." % container_image_name) def _upload_to_gcs(self, local_file_path, gcs_location): - bucket_name, blob_name = self._get_gcs_bucket_and_name(gcs_location) + gcs_bucket, gcs_object = self._get_gcs_bucket_and_name(gcs_location) + request = storage.StorageObjectsInsertRequest( + bucket=gcs_bucket, name=gcs_object) _LOGGER.info('Starting GCS upload to %s...', gcs_location) - from google.cloud import storage - from google.cloud.exceptions import Forbidden - from google.cloud.exceptions import NotFound + total_size = os.path.getsize(local_file_path) + from apitools.base.py import exceptions try: - bucket = self._storage_client.get_bucket(bucket_name) - blob = bucket.get_blob(blob_name) - if not blob: - blob = storage.Blob(name=blob_name, bucket=bucket) - blob.upload_from_filename(local_file_path) - except Exception as e: - if isinstance(e, (Forbidden, NotFound)): + with open(local_file_path, 'rb') as stream: + upload = storage.Upload(stream, 'application/octet-stream', total_size) + self._storage_client.objects.Insert(request, upload=upload) + except exceptions.HttpError as e: + reportable_errors = { + 403: 'access denied', + 404: 'bucket not found', + } + if e.status_code in reportable_errors: raise IOError(( 'Could not upload to GCS path %s: %s. Please verify ' 'that credentials are valid and that you have write ' - 'access to the specified path.') % (gcs_location, e.message)) + 'access to the specified path.') % + (gcs_location, reportable_errors[e.status_code])) raise _LOGGER.info('Completed GCS upload to %s.', gcs_location) diff --git a/sdks/python/mypy.ini b/sdks/python/mypy.ini index 298f249ffbff..46dea481f931 100644 --- a/sdks/python/mypy.ini +++ b/sdks/python/mypy.ini @@ -39,6 +39,9 @@ ignore_errors = true # error: Cannot infer type of lambda [misc] ignore_errors = true +[mypy-apache_beam.io.gcp.internal.clients.storage.storage_v1_client] +ignore_errors = true + [mypy-apache_beam.io.gcp.internal.clients.bigquery.bigquery_v2_client] ignore_errors = true diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 917e4eaba940..cadc4f34c86d 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -319,7 +319,6 @@ def get_portability_package_data(): 'google-cloud-datastore>=2.0.0,<3', 'google-cloud-pubsub>=2.1.0,<3', 'google-cloud-pubsublite>=1.2.0,<2', - 'google-cloud-storage>=2.10.0,<3', # GCP packages required by tests 'google-cloud-bigquery>=2.0.0,<4', 'google-cloud-bigquery-storage>=2.6.3,<3', From 170310ee83a11e0f4dd9010068be7ab08f1fc91c Mon Sep 17 00:00:00 2001 From: martin trieu Date: Thu, 28 Sep 2023 10:44:04 -0700 Subject: [PATCH 27/64] Break out nested classes from StreamingDataflowWorker. (#28537) Use ActiveWorkState class instead of an activeWorkMap in ComputationState --- .../dataflow/worker/BatchDataflowWorker.java | 95 +- .../worker/DataflowWorkUnitClient.java | 18 +- .../worker/StreamingDataflowWorker.java | 1004 +++---------- .../dataflow/worker/WorkUnitClient.java | 6 +- .../dataflow/worker/counters/NameContext.java | 7 +- .../worker/streaming/ActiveWorkState.java | 292 ++++ .../dataflow/worker/streaming/Commit.java | 43 + .../worker/streaming/ComputationState.java | 139 ++ .../worker/streaming/ExecutionState.java | 54 + .../streaming/KeyCommitTooLargeException.java | 50 + .../dataflow/worker/streaming/ShardedKey.java | 38 + .../dataflow/worker/streaming/StageInfo.java | 114 ++ .../streaming/WeightedBoundedQueue.java | 101 ++ .../dataflow/worker/streaming/Work.java | 173 +++ .../worker/BatchDataflowWorkerTest.java | 17 +- .../worker/DataflowWorkUnitClientTest.java | 31 +- .../worker/StreamingDataflowWorkerTest.java | 1303 ++++++++--------- .../worker/streaming/ActiveWorkStateTest.java | 296 ++++ .../streaming/WeightBoundedQueueTest.java | 194 +++ 19 files changed, 2422 insertions(+), 1553 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Commit.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/KeyCommitTooLargeException.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WeightedBoundedQueue.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/WeightBoundedQueueTest.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java index 9144729faca2..7407c97619b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java @@ -60,22 +60,8 @@ }) public class BatchDataflowWorker implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(BatchDataflowWorker.class); - - /** A client to get and update work items. */ - private final WorkUnitClient workUnitClient; - - /** - * Pipeline options, initially provided via the constructor and partially provided via each work - * work unit. - */ - private final DataflowWorkerHarnessOptions options; - - /** The factory to create {@link DataflowMapTaskExecutor DataflowMapTaskExecutors}. */ - private final DataflowMapTaskExecutorFactory mapTaskExecutorFactory; - /** The idGenerator to generate unique id globally. */ private static final IdGenerator idGenerator = IdGenerators.decrementingLongs(); - /** * Function which converts map tasks to their network representation for execution. * @@ -90,30 +76,7 @@ public class BatchDataflowWorker implements Closeable { new FixMultiOutputInfosOnParDoInstructions(idGenerator) .andThen(new MapTaskToNetworkFunction(idGenerator)); - /** Registry of known {@link ReaderFactory ReaderFactories}. */ - private final ReaderRegistry readerRegistry = ReaderRegistry.defaultRegistry(); - - /** Registry of known {@link SinkFactory SinkFactories}. */ - private final SinkRegistry sinkRegistry = SinkRegistry.defaultRegistry(); - - /** A side input cache shared between all execution contexts. */ - private final Cache> sideInputDataCache; - - /** - * A side input cache shared between all execution contexts. This cache is meant to store values - * as weak references. This allows for insertion of logical keys with zero weight since they will - * only be scoped to the lifetime of the value being cached. - */ - private final Cache sideInputWeakReferenceCache; - private static final int DEFAULT_STATUS_PORT = 8081; - - /** Status pages returning health of worker. */ - private WorkerStatusPages statusPages; - - /** Periodic sender of debug information to the debug capture service. */ - private DebugCapture.Manager debugCaptureManager = null; - /** * A weight in "bytes" for the overhead of a {@link Weighted} wrapper in the cache. It is just an * approximation so it is OK for it to be fairly arbitrary as long as it is nonzero. @@ -121,33 +84,42 @@ public class BatchDataflowWorker implements Closeable { private static final int OVERHEAD_WEIGHT = 8; private static final long MEGABYTES = 1024 * 1024; - /** * Limit the number of logical references. Weak references may never be cleared if the object is * long lived irrespective if the user actually is interested in the key lookup anymore. */ private static final int MAX_LOGICAL_REFERENCES = 1_000_000; - /** How many concurrent write operations to a cache should we allow. */ private static final int CACHE_CONCURRENCY_LEVEL = 4 * Runtime.getRuntime().availableProcessors(); + /** A client to get and update work items. */ + private final WorkUnitClient workUnitClient; + /** + * Pipeline options, initially provided via the constructor and partially provided via each work + * work unit. + */ + private final DataflowWorkerHarnessOptions options; + /** The factory to create {@link DataflowMapTaskExecutor DataflowMapTaskExecutors}. */ + private final DataflowMapTaskExecutorFactory mapTaskExecutorFactory; + /** Registry of known {@link ReaderFactory ReaderFactories}. */ + private final ReaderRegistry readerRegistry = ReaderRegistry.defaultRegistry(); + /** Registry of known {@link SinkFactory SinkFactories}. */ + private final SinkRegistry sinkRegistry = SinkRegistry.defaultRegistry(); + /** A side input cache shared between all execution contexts. */ + private final Cache> sideInputDataCache; + /** + * A side input cache shared between all execution contexts. This cache is meant to store values + * as weak references. This allows for insertion of logical keys with zero weight since they will + * only be scoped to the lifetime of the value being cached. + */ + private final Cache sideInputWeakReferenceCache; private final Function> mapTaskToNetwork; - private final MemoryMonitor memoryMonitor; private final Thread memoryMonitorThread; - - /** - * Returns a {@link BatchDataflowWorker} configured to execute user functions via intrinsic Java - * execution. - * - *

This is also known as the "legacy" or "pre-portability" approach. It is not yet deprecated - * as there is not a compatible path forward for users. - */ - static BatchDataflowWorker forBatchIntrinsicWorkerHarness( - WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) { - return new BatchDataflowWorker( - workUnitClient, IntrinsicMapTaskExecutorFactory.defaultFactory(), options); - } + /** Status pages returning health of worker. */ + private final WorkerStatusPages statusPages; + /** Periodic sender of debug information to the debug capture service. */ + private DebugCapture.Manager debugCaptureManager = null; protected BatchDataflowWorker( WorkUnitClient workUnitClient, @@ -188,6 +160,19 @@ protected BatchDataflowWorker( ExecutionStateSampler.instance().start(); } + /** + * Returns a {@link BatchDataflowWorker} configured to execute user functions via intrinsic Java + * execution. + * + *

This is also known as the "legacy" or "pre-portability" approach. It is not yet deprecated + * as there is not a compatible path forward for users. + */ + static BatchDataflowWorker forBatchIntrinsicWorkerHarness( + WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) { + return new BatchDataflowWorker( + workUnitClient, IntrinsicMapTaskExecutorFactory.defaultFactory(), options); + } + private static DebugCapture.Manager initializeAndStartDebugCaptureManager( DataflowWorkerHarnessOptions options, Collection debugCapturePages) { DebugCapture.Manager result = new DebugCapture.Manager(options, debugCapturePages); @@ -215,7 +200,7 @@ private static Thread startMemoryMonitorThread(MemoryMonitor memoryMonitor) { */ public boolean getAndPerformWork() throws IOException { while (true) { - Optional work = workUnitClient.getWorkItem(); + Optional work = Optional.fromJavaUtil(workUnitClient.getWorkItem()); if (work.isPresent()) { WorkItemStatusClient statusProvider = new WorkItemStatusClient(workUnitClient, work.get()); return doWork(work.get(), statusProvider); @@ -243,7 +228,7 @@ boolean doWork(WorkItem workItem, WorkItemStatusClient workItemStatusClient) thr } else if (workItem.getSourceOperationTask() != null) { stageName = workItem.getSourceOperationTask().getStageName(); } else { - throw new RuntimeException("Unknown kind of work item: " + workItem.toString()); + throw new RuntimeException("Unknown kind of work item: " + workItem); } CounterSet counterSet = new CounterSet(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java index bf809cfd0121..ffa377fd3f82 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java @@ -39,13 +39,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Optional; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.util.PropertyNames; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkProgressUpdater; import org.apache.beam.sdk.extensions.gcp.util.Transport; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.DateTime; @@ -87,7 +87,7 @@ class DataflowWorkUnitClient implements WorkUnitClient { } /** - * Gets a {@link WorkItem} from the Dataflow service, or returns {@link Optional#absent()} if no + * Gets a {@link WorkItem} from the Dataflow service, or returns {@link Optional#empty()} if no * work was found. * *

If work is returned, the calling thread should call reportWorkItemStatus after completing it @@ -116,11 +116,11 @@ public Optional getWorkItem() throws IOException { if (!workItem.isPresent()) { // Normal case, this means that the response contained no work, i.e. no work is available // at this time. - return Optional.absent(); + return Optional.empty(); } - if (workItem.isPresent() && workItem.get().getId() == null) { - logger.debug("Discarding invalid work item {}", workItem.orNull()); - return Optional.absent(); + if (workItem.get().getId() == null) { + logger.debug("Discarding invalid work item {}", workItem.get()); + return Optional.empty(); } WorkItem work = workItem.get(); @@ -148,7 +148,7 @@ public Optional getWorkItem() throws IOException { /** * Gets a global streaming config {@link WorkItem} from the Dataflow service, or returns {@link - * Optional#absent()} if no work was found. + * Optional#empty()} if no work was found. */ @Override public Optional getGlobalStreamingConfigWorkItem() throws IOException { @@ -158,7 +158,7 @@ public Optional getGlobalStreamingConfigWorkItem() throws IOException /** * Gets a streaming config {@link WorkItem} for the given computation from the Dataflow service, - * or returns {@link Optional#absent()} if no work was found. + * or returns {@link Optional#empty()} if no work was found. */ @Override public Optional getStreamingConfigWorkItem(String computationId) throws IOException { @@ -197,7 +197,7 @@ private Optional getWorkItemInternal( List workItems = response.getWorkItems(); if (workItems == null || workItems.isEmpty()) { // We didn't lease any work. - return Optional.absent(); + return Optional.empty(); } else if (workItems.size() > 1) { throw new IOException( "This version of the SDK expects no more than one work item from the service: " diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 8629b7116973..7110fee29362 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -19,10 +19,8 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.runners.dataflow.DataflowRunner.hasExperiment; -import static org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.THROTTLING_MSECS_METRIC_NAME; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.api.services.dataflow.model.CounterStructuredName; import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.MapTask; import com.google.api.services.dataflow.model.Status; @@ -30,23 +28,19 @@ import com.google.api.services.dataflow.model.StreamingConfigTask; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemStatus; -import com.google.auto.value.AutoValue; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.File; import java.io.IOException; import java.io.PrintWriter; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Deque; -import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Queue; +import java.util.Optional; import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -54,9 +48,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -66,17 +58,13 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import org.apache.beam.runners.core.metrics.ExecutionStateSampler; -import org.apache.beam.runners.core.metrics.ExecutionStateTracker; import org.apache.beam.runners.core.metrics.MetricsLogger; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.internal.CustomSources; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.util.CloudObject; import org.apache.beam.runners.dataflow.util.CloudObjects; -import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingPerStageSystemCounterNames; import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingSystemCounterNames; -import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.Work.State; -import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionStateRegistry; import org.apache.beam.runners.dataflow.worker.apiary.FixMultiOutputInfosOnParDoInstructions; import org.apache.beam.runners.dataflow.worker.counters.Counter; import org.apache.beam.runners.dataflow.worker.counters.CounterSet; @@ -97,6 +85,15 @@ import org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages; +import org.apache.beam.runners.dataflow.worker.streaming.Commit; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.ExecutionState; +import org.apache.beam.runners.dataflow.worker.streaming.KeyCommitTooLargeException; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; +import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.Work.State; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter; @@ -125,9 +122,7 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; @@ -154,8 +149,29 @@ }) public class StreamingDataflowWorker { - private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); + // TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic + // throttling-msecs metric. + public static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME = + MetricName.named( + "org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl", + "throttling-msecs"); + // Maximum number of threads for processing. Currently each thread processes one key at a time. + static final int MAX_PROCESSING_THREADS = 300; + static final long THREAD_EXPIRATION_TIME_SEC = 60; + static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20; + static final int MAX_COMMIT_QUEUE_BYTES = 500 << 20; // 500MB + static final int NUM_COMMIT_STREAMS = 1; + static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; + static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); + /** + * Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked + * (across all the sinks, if there are more than one) reaches this limit. This serves as hint for + * readers to stop producing more. This can be disabled with 'disable_limiting_bundle_sink_bytes' + * experiment. + */ + static final int MAX_SINK_BYTES = 10_000_000; + private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); /** The idGenerator to generate unique id globally. */ private static final IdGenerator idGenerator = IdGenerators.decrementingLongs(); /** @@ -164,7 +180,6 @@ public class StreamingDataflowWorker { */ private static final Function fixMultiOutputInfos = new FixMultiOutputInfosOnParDoInstructions(idGenerator); - /** * Function which converts map tasks to their network representation for execution. * @@ -176,235 +191,35 @@ public class StreamingDataflowWorker { private static final Function> mapTaskToBaseNetwork = new MapTaskToNetworkFunction(idGenerator); - private static Random clientIdGenerator = new Random(); - - // Maximum number of threads for processing. Currently each thread processes one key at a time. - static final int MAX_PROCESSING_THREADS = 300; - static final long THREAD_EXPIRATION_TIME_SEC = 60; - static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20; - static final int MAX_COMMIT_QUEUE_BYTES = 500 << 20; // 500MB - static final int NUM_COMMIT_STREAMS = 1; - static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; - static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); - private static final int DEFAULT_STATUS_PORT = 8081; - // Maximum size of the result of a GetWork request. private static final long MAX_GET_WORK_FETCH_BYTES = 64L << 20; // 64m - // Reserved ID for counter updates. // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc. private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3"; - /** Maximum number of failure stacktraces to report in each update sent to backend. */ private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000; - // TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic - // throttling-msecs metric. - public static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME = - MetricName.named( - "org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl", - "throttling-msecs"); - private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION = Duration.standardMinutes(5); - - /** Returns whether an exception was caused by a {@link OutOfMemoryError}. */ - private static boolean isOutOfMemoryError(Throwable t) { - while (t != null) { - if (t instanceof OutOfMemoryError) { - return true; - } - t = t.getCause(); - } - return false; - } - - private static class KeyCommitTooLargeException extends Exception { - - public static KeyCommitTooLargeException causedBy( - String computationId, long byteLimit, WorkItemCommitRequest request) { - StringBuilder message = new StringBuilder(); - message.append("Commit request for stage "); - message.append(computationId); - message.append(" and key "); - message.append(request.getKey().toStringUtf8()); - if (request.getSerializedSize() > 0) { - message.append( - " has size " - + request.getSerializedSize() - + " which is more than the limit of " - + byteLimit); - } else { - message.append(" is larger than 2GB and cannot be processed"); - } - message.append( - ". This may be caused by grouping a very " - + "large amount of data in a single window without using Combine," - + " or by producing a large amount of data from a single input element."); - return new KeyCommitTooLargeException(message.toString()); - } - - private KeyCommitTooLargeException(String message) { - super(message); - } - } - - private static MapTask parseMapTask(String input) throws IOException { - return Transport.getJsonFactory().fromString(input, MapTask.class); - } - - public static void main(String[] args) throws Exception { - JvmInitializers.runOnStartup(); - - DataflowWorkerHarnessHelper.initializeLogging(StreamingDataflowWorker.class); - DataflowWorkerHarnessOptions options = - DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( - StreamingDataflowWorker.class); - DataflowWorkerHarnessHelper.configureLogging(options); - checkArgument( - options.isStreaming(), - "%s instantiated with options indicating batch use", - StreamingDataflowWorker.class.getName()); - - checkArgument( - !DataflowRunner.hasExperiment(options, "beam_fn_api"), - "%s cannot be main() class with beam_fn_api enabled", - StreamingDataflowWorker.class.getSimpleName()); - - StreamingDataflowWorker worker = - StreamingDataflowWorker.fromDataflowWorkerHarnessOptions(options); - - // Use the MetricsLogger container which is used by BigQueryIO to periodically log process-wide - // metrics. - MetricsEnvironment.setProcessWideContainer(new MetricsLogger(null)); - - JvmInitializers.runBeforeProcessing(options); - worker.startStatusPages(); - worker.start(); - } - - /** Bounded set of queues, with a maximum total weight. */ - private static class WeightedBoundedQueue { - - private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); - private final int maxWeight; - private final Semaphore limit; - private final Function weigher; - - public WeightedBoundedQueue(int maxWeight, Function weigher) { - this.maxWeight = maxWeight; - this.limit = new Semaphore(maxWeight, true); - this.weigher = weigher; - } - - /** - * Adds the value to the queue, blocking if this would cause the overall weight to exceed the - * limit. - */ - public void put(V value) { - limit.acquireUninterruptibly(weigher.apply(value)); - queue.add(value); - } - - /** Returns and removes the next value, or null if there is no such value. */ - public @Nullable V poll() { - V result = queue.poll(); - if (result != null) { - limit.release(weigher.apply(result)); - } - return result; - } - - /** - * Retrieves and removes the head of this queue, waiting up to the specified wait time if - * necessary for an element to become available. - * - * @param timeout how long to wait before giving up, in units of {@code unit} - * @param unit a {@code TimeUnit} determining how to interpret the {@code timeout} parameter - * @return the head of this queue, or {@code null} if the specified waiting time elapses before - * an element is available - * @throws InterruptedException if interrupted while waiting - */ - public @Nullable V poll(long timeout, TimeUnit unit) throws InterruptedException { - V result = queue.poll(timeout, unit); - if (result != null) { - limit.release(weigher.apply(result)); - } - return result; - } - - /** Returns and removes the next value, or blocks until one is available. */ - public @Nullable V take() throws InterruptedException { - V result = queue.take(); - limit.release(weigher.apply(result)); - return result; - } - - /** Returns the current weight of the queue. */ - public int weight() { - return maxWeight - limit.availablePermits(); - } - - public int size() { - return queue.size(); - } - } - - // Value class for a queued commit. - static class Commit { - - private Windmill.WorkItemCommitRequest request; - private ComputationState computationState; - private Work work; - - public Commit( - Windmill.WorkItemCommitRequest request, ComputationState computationState, Work work) { - this.request = request; - assert request.getSerializedSize() > 0; - this.computationState = computationState; - this.work = work; - } - - public Windmill.WorkItemCommitRequest getRequest() { - return request; - } - - public ComputationState getComputationState() { - return computationState; - } - - public Work getWork() { - return work; - } - - public int getSize() { - return request.getSerializedSize(); - } - } - + private static final Random clientIdGenerator = new Random(); + final WindmillStateCache stateCache; // Maps from computation ids to per-computation state. private final ConcurrentMap computationMap = new ConcurrentHashMap<>(); private final WeightedBoundedQueue commitQueue = - new WeightedBoundedQueue<>( + WeightedBoundedQueue.create( MAX_COMMIT_QUEUE_BYTES, commit -> Math.min(MAX_COMMIT_QUEUE_BYTES, commit.getSize())); - // Cache of tokens to commit callbacks. // Using Cache with time eviction policy helps us to prevent memory leak when callback ids are // discarded by Dataflow service and calling commitCallback is best-effort. private final Cache commitCallbacks = CacheBuilder.newBuilder().expireAfterWrite(5L, TimeUnit.MINUTES).build(); - // Map of user state names to system state names. // TODO(drieber): obsolete stateNameMap. Use transformUserNameToStateFamily in // ComputationState instead. private final ConcurrentMap stateNameMap = new ConcurrentHashMap<>(); private final ConcurrentMap systemNameToComputationIdMap = new ConcurrentHashMap<>(); - - final WindmillStateCache stateCache; - private final ThreadFactory threadFactory; - private DataflowMapTaskExecutorFactory mapTaskExecutorFactory; private final BoundedQueueExecutor workUnitExecutor; private final WindmillServerStub windmillServer; private final Thread dispatchThread; @@ -415,16 +230,13 @@ public int getSize() { private final StreamingDataflowWorkerOptions options; private final boolean windmillServiceEnabled; private final long clientId; - private final MetricTrackingWindmillServerStub metricTrackingWindmillServer; private final CounterSet pendingDeltaCounters = new CounterSet(); private final CounterSet pendingCumulativeCounters = new CounterSet(); private final java.util.concurrent.ConcurrentLinkedQueue pendingMonitoringInfos = new ConcurrentLinkedQueue<>(); - // Map from stage name to StageInfo containing metrics container registry and per stage counters. private final ConcurrentMap stageInfoMap = new ConcurrentHashMap(); - // Built-in delta counters. private final Counter windmillShuffleBytesRead; private final Counter windmillStateBytesRead; @@ -436,134 +248,35 @@ public int getSize() { private final Counter timeAtMaxActiveThreads; private final Counter windmillMaxObservedWorkItemCommitBytes; private final Counter memoryThrashing; - private ScheduledExecutorService refreshWorkTimer; - private ScheduledExecutorService statusPageTimer; - private final boolean publishCounters; - private ScheduledExecutorService globalWorkerUpdatesTimer; - private int retryLocallyDelayMs = 10000; - - // Periodically fires a global config request to dataflow service. Only used when windmill service - // is enabled. - private ScheduledExecutorService globalConfigRefreshTimer; - private final MemoryMonitor memoryMonitor; private final Thread memoryMonitorThread; - private final WorkerStatusPages statusPages; - // Periodic sender of debug information to the debug capture service. - private DebugCapture.Manager debugCaptureManager = null; - // Limit on bytes sinked (committed) in a work item. private final long maxSinkBytes; // = MAX_SINK_BYTES unless disabled in options. - // Possibly overridden by streaming engine config. - private int maxWorkItemCommitBytes = Integer.MAX_VALUE; - private final EvictingQueue pendingFailuresToReport = - EvictingQueue.create(MAX_FAILURES_TO_REPORT_IN_UPDATE); - + EvictingQueue.create(MAX_FAILURES_TO_REPORT_IN_UPDATE); private final ReaderCache readerCache; - private final WorkUnitClient workUnitClient; private final CompletableFuture isDoneFuture; private final Function> mapTaskToNetwork; - - /** - * Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked - * (across all the sinks, if there are more than one) reaches this limit. This serves as hint for - * readers to stop producing more. This can be disabled with 'disable_limiting_bundle_sink_bytes' - * experiment. - */ - static final int MAX_SINK_BYTES = 10_000_000; - private final ReaderRegistry readerRegistry = ReaderRegistry.defaultRegistry(); private final SinkRegistry sinkRegistry = SinkRegistry.defaultRegistry(); - - private HotKeyLogger hotKeyLogger; - private final Supplier clock; private final Function executorSupplier; - - /** Contains a few of the stage specific fields. E.g. metrics container registry, counters etc. */ - private static class StageInfo { - - final String stageName; - final String systemName; - final MetricsContainerRegistry metricsContainerRegistry; - final StreamingModeExecutionStateRegistry executionStateRegistry; - final CounterSet deltaCounters; - final Counter throttledMsecs; - final Counter totalProcessingMsecs; - final Counter timerProcessingMsecs; - - StageInfo(String stageName, String systemName, StreamingDataflowWorker worker) { - this.stageName = stageName; - this.systemName = systemName; - metricsContainerRegistry = StreamingStepMetricsContainer.createRegistry(); - executionStateRegistry = new StreamingModeExecutionStateRegistry(worker); - NameContext nameContext = NameContext.create(stageName, null, systemName, null); - deltaCounters = new CounterSet(); - throttledMsecs = - deltaCounters.longSum( - StreamingPerStageSystemCounterNames.THROTTLED_MSECS.counterName(nameContext)); - totalProcessingMsecs = - deltaCounters.longSum( - StreamingPerStageSystemCounterNames.TOTAL_PROCESSING_MSECS.counterName(nameContext)); - timerProcessingMsecs = - deltaCounters.longSum( - StreamingPerStageSystemCounterNames.TIMER_PROCESSING_MSECS.counterName(nameContext)); - } - - List extractCounterUpdates() { - List counterUpdates = new ArrayList<>(); - Iterables.addAll( - counterUpdates, - StreamingStepMetricsContainer.extractMetricUpdates(metricsContainerRegistry)); - Iterables.addAll(counterUpdates, executionStateRegistry.extractUpdates(false)); - for (CounterUpdate counterUpdate : counterUpdates) { - translateKnownStepCounters(counterUpdate); - } - counterUpdates.addAll( - deltaCounters.extractModifiedDeltaUpdates(DataflowCounterUpdateExtractor.INSTANCE)); - return counterUpdates; - } - - // Checks if the step counter affects any per-stage counters. Currently 'throttled_millis' - // is the only counter updated. - private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) { - CounterStructuredName structuredName = - stepCounterUpdate.getStructuredNameAndMetadata().getName(); - if ((THROTTLING_MSECS_METRIC_NAME.getNamespace().equals(structuredName.getOriginNamespace()) - && THROTTLING_MSECS_METRIC_NAME.getName().equals(structuredName.getName())) - || (BIGQUERY_STREAMING_INSERT_THROTTLE_TIME - .getNamespace() - .equals(structuredName.getOriginNamespace()) - && BIGQUERY_STREAMING_INSERT_THROTTLE_TIME - .getName() - .equals(structuredName.getName()))) { - long msecs = DataflowCounterUpdateExtractor.splitIntToLong(stepCounterUpdate.getInteger()); - if (msecs > 0) { - throttledMsecs.addValue(msecs); - } - } - } - } - - public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions( - DataflowWorkerHarnessOptions options) throws IOException { - - return new StreamingDataflowWorker( - Collections.emptyList(), - IntrinsicMapTaskExecutorFactory.defaultFactory(), - new DataflowWorkUnitClient(options, LOG), - options.as(StreamingDataflowWorkerOptions.class), - true, - new HotKeyLogger(), - Instant::now, - (threadName) -> - Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat(threadName).build())); - } + private final DataflowMapTaskExecutorFactory mapTaskExecutorFactory; + private final HotKeyLogger hotKeyLogger; + // Periodic sender of debug information to the debug capture service. + private final DebugCapture.@Nullable Manager debugCaptureManager; + private ScheduledExecutorService refreshWorkTimer; + private ScheduledExecutorService statusPageTimer; + private ScheduledExecutorService globalWorkerUpdatesTimer; + private int retryLocallyDelayMs = 10000; + // Periodically fires a global config request to dataflow service. Only used when windmill service + // is enabled. + private ScheduledExecutorService globalConfigRefreshTimer; + // Possibly overridden by streaming engine config. + private int maxWorkItemCommitBytes = Integer.MAX_VALUE; @VisibleForTesting StreamingDataflowWorker( @@ -593,6 +306,8 @@ public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions( if (windmillServiceEnabled) { this.debugCaptureManager = new DebugCapture.Manager(options, statusPages.getDebugCapturePages()); + } else { + this.debugCaptureManager = null; } this.windmillShuffleBytesRead = pendingDeltaCounters.longSum( @@ -701,6 +416,81 @@ public void run() { LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes); } + /** Returns whether an exception was caused by a {@link OutOfMemoryError}. */ + private static boolean isOutOfMemoryError(Throwable t) { + while (t != null) { + if (t instanceof OutOfMemoryError) { + return true; + } + t = t.getCause(); + } + return false; + } + + private static MapTask parseMapTask(String input) throws IOException { + return Transport.getJsonFactory().fromString(input, MapTask.class); + } + + public static void main(String[] args) throws Exception { + JvmInitializers.runOnStartup(); + + DataflowWorkerHarnessHelper.initializeLogging(StreamingDataflowWorker.class); + DataflowWorkerHarnessOptions options = + DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( + StreamingDataflowWorker.class); + DataflowWorkerHarnessHelper.configureLogging(options); + checkArgument( + options.isStreaming(), + "%s instantiated with options indicating batch use", + StreamingDataflowWorker.class.getName()); + + checkArgument( + !DataflowRunner.hasExperiment(options, "beam_fn_api"), + "%s cannot be main() class with beam_fn_api enabled", + StreamingDataflowWorker.class.getSimpleName()); + + StreamingDataflowWorker worker = + StreamingDataflowWorker.fromDataflowWorkerHarnessOptions(options); + + // Use the MetricsLogger container which is used by BigQueryIO to periodically log process-wide + // metrics. + MetricsEnvironment.setProcessWideContainer(new MetricsLogger(null)); + + JvmInitializers.runBeforeProcessing(options); + worker.startStatusPages(); + worker.start(); + } + + public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions( + DataflowWorkerHarnessOptions options) throws IOException { + + return new StreamingDataflowWorker( + Collections.emptyList(), + IntrinsicMapTaskExecutorFactory.defaultFactory(), + new DataflowWorkUnitClient(options, LOG), + options.as(StreamingDataflowWorkerOptions.class), + true, + new HotKeyLogger(), + Instant::now, + (threadName) -> + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat(threadName).build())); + } + + private static void sleep(int millis) { + Uninterruptibles.sleepUninterruptibly(millis, TimeUnit.MILLISECONDS); + } + + /** Sets the stage name and workId of the current Thread for logging. */ + private static void setUpWorkLoggingContext(Windmill.WorkItem workItem, String computationId) { + String workIdBuilder = + Long.toHexString(workItem.getShardingKey()) + + '-' + + Long.toHexString(workItem.getWorkToken()); + DataflowWorkerLoggingMDC.setWorkId(workIdBuilder); + DataflowWorkerLoggingMDC.setStageName(computationId); + } + private int chooseMaximumNumberOfThreads() { if (options.getNumberOfWorkerHarnessThreads() != 0) { return options.getNumberOfWorkerHarnessThreads(); @@ -810,7 +600,7 @@ public void run() { + options.getWorkerId() + "_" + page.pageName() - + timestamp.toString()) + + timestamp) .replaceAll("/", "_")); writer = new PrintWriter(outputFile, UTF_8.name()); page.captureData(writer); @@ -938,10 +728,6 @@ private synchronized void addComputation( } } - private static void sleep(int millis) { - Uninterruptibles.sleepUninterruptibly(millis, TimeUnit.MILLISECONDS); - } - /** * If the computation is not yet known about, configuration for it will be fetched. This can still * return null if there is no configuration fetched for the computation. @@ -996,7 +782,7 @@ private void dispatchLoop() { inputDataWatermark, synchronizedProcessingTime, workItem, - /*getWorkStreamLatencies=*/ Collections.emptyList()); + /* getWorkStreamLatencies= */ Collections.emptyList()); } } } @@ -1049,138 +835,20 @@ private void scheduleWorkItem( WindmillTimeUtils.windmillToHarnessWatermark(workItem.getOutputDataWatermark()); Preconditions.checkState( outputDataWatermark == null || !outputDataWatermark.isAfter(inputDataWatermark)); - Work work = - new Work(workItem, clock, getWorkStreamLatencies) { - @Override - public void run() { - process( - computationState, - inputDataWatermark, - outputDataWatermark, - synchronizedProcessingTime, - this); - } - }; + Work scheduledWork = + Work.create( + workItem, + clock, + getWorkStreamLatencies, + work -> + process( + computationState, + inputDataWatermark, + outputDataWatermark, + synchronizedProcessingTime, + work)); computationState.activateWork( - ShardedKey.create(workItem.getKey(), workItem.getShardingKey()), work); - } - - @AutoValue - abstract static class ShardedKey { - - public static ShardedKey create(ByteString key, long shardingKey) { - return new AutoValue_StreamingDataflowWorker_ShardedKey(key, shardingKey); - } - - public abstract ByteString key(); - - public abstract long shardingKey(); - - @Override - public final String toString() { - ByteString keyToDisplay = key(); - if (keyToDisplay.size() > 100) { - keyToDisplay = keyToDisplay.substring(0, 100); - } - return String.format("%016x-%s", shardingKey(), TextFormat.escapeBytes(keyToDisplay)); - } - } - - abstract static class Work implements Runnable { - - enum State { - QUEUED(Windmill.LatencyAttribution.State.QUEUED), - PROCESSING(Windmill.LatencyAttribution.State.ACTIVE), - READING(Windmill.LatencyAttribution.State.READING), - COMMIT_QUEUED(Windmill.LatencyAttribution.State.COMMITTING), - COMMITTING(Windmill.LatencyAttribution.State.COMMITTING), - GET_WORK_IN_WINDMILL_WORKER(Windmill.LatencyAttribution.State.GET_WORK_IN_WINDMILL_WORKER), - GET_WORK_IN_TRANSIT_TO_DISPATCHER( - Windmill.LatencyAttribution.State.GET_WORK_IN_TRANSIT_TO_DISPATCHER), - GET_WORK_IN_TRANSIT_TO_USER_WORKER( - Windmill.LatencyAttribution.State.GET_WORK_IN_TRANSIT_TO_USER_WORKER); - - private final Windmill.LatencyAttribution.State latencyAttributionState; - - private State(Windmill.LatencyAttribution.State latencyAttributionState) { - this.latencyAttributionState = latencyAttributionState; - } - - Windmill.LatencyAttribution.State toLatencyAttributionState() { - return latencyAttributionState; - } - } - - private final Windmill.WorkItem workItem; - private final Supplier clock; - private final Instant startTime; - private Instant stateStartTime; - private State state; - private final Map totalDurationPerState = - new EnumMap<>(Windmill.LatencyAttribution.State.class); - - public Work( - Windmill.WorkItem workItem, - Supplier clock, - Collection getWorkStreamLatencies) { - this.workItem = workItem; - this.clock = clock; - this.startTime = this.stateStartTime = clock.get(); - this.state = State.QUEUED; - recordGetWorkStreamLatencies(getWorkStreamLatencies); - } - - public Windmill.WorkItem getWorkItem() { - return workItem; - } - - public Instant getStartTime() { - return startTime; - } - - public State getState() { - return state; - } - - public void setState(State state) { - Instant now = clock.get(); - totalDurationPerState.compute( - this.state.toLatencyAttributionState(), - (s, d) -> new Duration(this.stateStartTime, now).plus(d == null ? Duration.ZERO : d)); - this.state = state; - this.stateStartTime = now; - } - - public Instant getStateStartTime() { - return stateStartTime; - } - - private void recordGetWorkStreamLatencies( - Collection getWorkStreamLatencies) { - for (LatencyAttribution latency : getWorkStreamLatencies) { - totalDurationPerState.put( - latency.getState(), Duration.millis(latency.getTotalDurationMillis())); - } - } - - public Collection getLatencyAttributions() { - List list = new ArrayList<>(); - for (Windmill.LatencyAttribution.State state : Windmill.LatencyAttribution.State.values()) { - Duration duration = totalDurationPerState.getOrDefault(state, Duration.ZERO); - if (state == this.state.toLatencyAttributionState()) { - duration = duration.plus(new Duration(this.stateStartTime, clock.get())); - } - if (duration.equals(Duration.ZERO)) { - continue; - } - list.add( - Windmill.LatencyAttribution.newBuilder() - .setState(state) - .setTotalDurationMillis(duration.getMillis()) - .build()); - } - return list; - } + ShardedKey.create(workItem.getKey(), workItem.getShardingKey()), scheduledWork); } /** @@ -1250,15 +918,9 @@ private void process( final String computationId = computationState.getComputationId(); final ByteString key = workItem.getKey(); work.setState(State.PROCESSING); - { - StringBuilder workIdBuilder = new StringBuilder(33); - workIdBuilder.append(Long.toHexString(workItem.getShardingKey())); - workIdBuilder.append('-'); - workIdBuilder.append(Long.toHexString(workItem.getWorkToken())); - DataflowWorkerLoggingMDC.setWorkId(workIdBuilder.toString()); - } - DataflowWorkerLoggingMDC.setStageName(computationId); + setUpWorkLoggingContext(workItem, computationId); + LOG.debug("Starting processing for {}:\n{}", computationId, work); Windmill.WorkItemCommitRequest.Builder outputBuilder = initializeOutputBuilder(key, workItem); @@ -1269,7 +931,7 @@ private void process( if (workItem.getSourceState().getOnlyFinalize()) { outputBuilder.setSourceStateUpdates(Windmill.SourceState.newBuilder().setOnlyFinalize(true)); work.setState(State.COMMIT_QUEUED); - commitQueue.put(new Commit(outputBuilder.build(), computationState, work)); + commitQueue.put(Commit.create(outputBuilder.build(), computationState, work)); return; } @@ -1279,7 +941,7 @@ private void process( StageInfo stageInfo = stageInfoMap.computeIfAbsent( - mapTask.getStageName(), s -> new StageInfo(s, mapTask.getSystemName(), this)); + mapTask.getStageName(), s -> StageInfo.create(s, mapTask.getSystemName(), this)); ExecutionState executionState = null; String counterName = "dataflow_source_bytes_processed-" + mapTask.getSystemName(); @@ -1304,12 +966,14 @@ private void process( DataflowExecutionContext.DataflowExecutionStateTracker executionStateTracker = new DataflowExecutionContext.DataflowExecutionStateTracker( ExecutionStateSampler.instance(), - stageInfo.executionStateRegistry.getState( - NameContext.forStage(mapTask.getStageName()), - "other", - null, - ScopedProfiler.INSTANCE.emptyScope()), - stageInfo.deltaCounters, + stageInfo + .executionStateRegistry() + .getState( + NameContext.forStage(mapTask.getStageName()), + "other", + null, + ScopedProfiler.INSTANCE.emptyScope()), + stageInfo.deltaCounters(), options, computationId); StreamingModeExecutionContext context = @@ -1321,9 +985,9 @@ private void process( ? computationState.getTransformUserNameToStateFamily() : stateNameMap, stateCache.forComputation(computationId), - stageInfo.metricsContainerRegistry, + stageInfo.metricsContainerRegistry(), executionStateTracker, - stageInfo.executionStateRegistry, + stageInfo.executionStateRegistry(), maxSinkBytes); DataflowMapTaskExecutor mapTaskExecutor = mapTaskExecutorFactory.create( @@ -1370,8 +1034,18 @@ private void process( .setSamplingPeriod(100) .countBytes(counterName)); } - executionState = - new ExecutionState(mapTaskExecutor, context, keyCoder, executionStateTracker); + + ExecutionState.Builder executionStateBuilder = + ExecutionState.builder() + .setWorkExecutor(mapTaskExecutor) + .setContext(context) + .setExecutionStateTracker(executionStateTracker); + + if (keyCoder != null) { + executionStateBuilder.setKeyCoder(keyCoder); + } + + executionState = executionStateBuilder.build(); } WindmillStateReader stateReader = @@ -1398,10 +1072,10 @@ public void close() { // // The coder type that will be present is: // WindowedValueCoder(TimerOrElementCoder(KvCoder)) - @Nullable Coder keyCoder = executionState.getKeyCoder(); + Optional> keyCoder = executionState.keyCoder(); @Nullable Object executionKey = - keyCoder == null ? null : keyCoder.decode(key.newInput(), Coder.Context.OUTER); + !keyCoder.isPresent() ? null : keyCoder.get().decode(key.newInput(), Coder.Context.OUTER); if (workItem.hasHotKeyInfo()) { Windmill.HotKeyInfo hotKeyInfo = workItem.getHotKeyInfo(); @@ -1410,7 +1084,7 @@ public void close() { // The MapTask instruction is ordered by dependencies, such that the first element is // always going to be the shuffle task. String stepName = computationState.getMapTask().getInstructions().get(0).getName(); - if (options.isHotKeyLoggingEnabled() && keyCoder != null) { + if (options.isHotKeyLoggingEnabled() && keyCoder.isPresent()) { hotKeyLogger.logHotKeyDetection(stepName, hotKeyAge, executionKey); } else { hotKeyLogger.logHotKeyDetection(stepName, hotKeyAge); @@ -1418,7 +1092,7 @@ public void close() { } executionState - .getContext() + .context() .start( executionKey, workItem, @@ -1430,13 +1104,13 @@ public void close() { outputBuilder); // Blocks while executing work. - executionState.getWorkExecutor().execute(); + executionState.workExecutor().execute(); // Reports source bytes processed to workitemcommitrequest if available. try { long sourceBytesProcessed = 0; HashMap counters = - ((DataflowMapTaskExecutor) executionState.getWorkExecutor()) + ((DataflowMapTaskExecutor) executionState.workExecutor()) .getReadOperation() .receivers[0] .getOutputCounters(); @@ -1450,9 +1124,9 @@ public void close() { } Iterables.addAll( - this.pendingMonitoringInfos, executionState.getWorkExecutor().extractMetricUpdates()); + this.pendingMonitoringInfos, executionState.workExecutor().extractMetricUpdates()); - commitCallbacks.putAll(executionState.getContext().flushState()); + commitCallbacks.putAll(executionState.context().flushState()); // Release the execution state for another thread to use. computationState.getExecutionStateQueue().offer(executionState); @@ -1481,7 +1155,7 @@ public void close() { commitRequest = buildWorkItemTruncationRequest(key, workItem, estimatedCommitSize); } - commitQueue.put(new Commit(commitRequest, computationState, work)); + commitQueue.put(Commit.create(commitRequest, computationState, work)); // Compute shuffle and state byte statistics these will be flushed asynchronously. long stateBytesWritten = @@ -1505,8 +1179,8 @@ public void close() { } catch (Throwable t) { if (executionState != null) { try { - executionState.getContext().invalidateCache(); - executionState.getWorkExecutor().close(); + executionState.context().invalidateCache(); + executionState.workExecutor().close(); } catch (Exception e) { LOG.warn("Failed to close map task executor: ", e); } finally { @@ -1571,7 +1245,7 @@ public void close() { } else { // Consider the item invalid. It will eventually be retried by Windmill if it still needs to // be processed. - computationState.completeWork( + computationState.completeWorkAndScheduleNextWorkForKey( ShardedKey.create(key, workItem.getShardingKey()), workItem.getWorkToken()); } } finally { @@ -1579,7 +1253,7 @@ public void close() { // work items causing exceptions are also accounted in time spent. long processingTimeMsecs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - processingStartTimeNanos); - stageInfo.totalProcessingMsecs.addValue(processingTimeMsecs); + stageInfo.totalProcessingMsecs().addValue(processingTimeMsecs); // Attribute all the processing to timers if the work item contains any timers. // Tests show that work items rarely contain both timers and message bundles. It should @@ -1587,7 +1261,7 @@ public void close() { // Another option: Derive time split between messages and timers based on recent totals. // either here or in DFE. if (work.getWorkItem().hasTimers()) { - stageInfo.timerProcessingMsecs.addValue(processingTimeMsecs); + stageInfo.timerProcessingMsecs().addValue(processingTimeMsecs); } DataflowWorkerLoggingMDC.setWorkId(null); @@ -1620,8 +1294,8 @@ private void commitLoop() { continue; } while (commit != null) { - ComputationState computationState = commit.getComputationState(); - commit.getWork().setState(State.COMMITTING); + ComputationState computationState = commit.computationState(); + commit.work().setState(Work.State.COMMITTING); Windmill.ComputationCommitWorkRequest.Builder computationRequestBuilder = computationRequestMap.get(computationState); if (computationRequestBuilder == null) { @@ -1629,7 +1303,7 @@ private void commitLoop() { computationRequestBuilder.setComputationId(computationState.getComputationId()); computationRequestMap.put(computationState, computationRequestBuilder); } - computationRequestBuilder.addRequests(commit.getRequest()); + computationRequestBuilder.addRequests(commit.request()); // Send the request if we've exceeded the bytes or there is no more // pending work. commitBytes is a long, so this cannot overflow. commitBytes += commit.getSize(); @@ -1647,7 +1321,7 @@ private void commitLoop() { computationRequestMap.entrySet()) { ComputationState computationState = entry.getKey(); for (Windmill.WorkItemCommitRequest workRequest : entry.getValue().getRequestsList()) { - computationState.completeWork( + computationState.completeWorkAndScheduleNextWorkForKey( ShardedKey.create(workRequest.getKey(), workRequest.getShardingKey()), workRequest.getWorkToken()); } @@ -1658,34 +1332,34 @@ private void commitLoop() { // Adds the commit to the commitStream if it fits, returning true iff it is consumed. private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { Preconditions.checkNotNull(commit); - final ComputationState state = commit.getComputationState(); - final Windmill.WorkItemCommitRequest request = commit.getRequest(); + final ComputationState state = commit.computationState(); + final Windmill.WorkItemCommitRequest request = commit.request(); final int size = commit.getSize(); - commit.getWork().setState(State.COMMITTING); + commit.work().setState(Work.State.COMMITTING); activeCommitBytes.addAndGet(size); if (commitStream.commitWorkItem( - state.computationId, + state.getComputationId(), request, (Windmill.CommitStatus status) -> { if (status != Windmill.CommitStatus.OK) { readerCache.invalidateReader( WindmillComputationKey.create( - state.computationId, request.getKey(), request.getShardingKey())); + state.getComputationId(), request.getKey(), request.getShardingKey())); stateCache - .forComputation(state.computationId) + .forComputation(state.getComputationId()) .invalidate(request.getKey(), request.getShardingKey()); } activeCommitBytes.addAndGet(-size); // This may throw an exception if the commit was not active, which is possible if it // was deemed stuck. - state.completeWork( + state.completeWorkAndScheduleNextWorkForKey( ShardedKey.create(request.getKey(), request.getShardingKey()), request.getWorkToken()); })) { return true; } else { // Back out the stats changes since the commit wasn't consumed. - commit.getWork().setState(State.COMMIT_QUEUED); + commit.work().setState(Work.State.COMMIT_QUEUED); activeCommitBytes.addAndGet(-size); return false; } @@ -1699,7 +1373,7 @@ private Commit batchCommitsToStream(CommitWorkStream commitStream) { Commit commit; try { if (commits < 5) { - commit = commitQueue.poll(10 - 2 * commits, TimeUnit.MILLISECONDS); + commit = commitQueue.poll(10 - 2L * commits, TimeUnit.MILLISECONDS); } else { commit = commitQueue.poll(); } @@ -1786,7 +1460,8 @@ private void getConfigFromWindmill(String computation) { addComputation( computationId, mapTask, - transformUserNameToStateFamilyByComputationId.get(computationId)); + transformUserNameToStateFamilyByComputationId.getOrDefault( + computationId, ImmutableMap.of())); } catch (IOException e) { LOG.warn("Parsing MapTask failed: {}", serializedMapTask); LOG.warn("Error: ", e); @@ -1804,13 +1479,12 @@ private void getConfigFromWindmill(String computation) { * @throws IOException if the RPC fails. */ private void getConfigFromDataflowService(@Nullable String computation) throws IOException { - Optional workItem; - if (computation != null) { - workItem = workUnitClient.getStreamingConfigWorkItem(computation); - } else { - workItem = workUnitClient.getGlobalStreamingConfigWorkItem(); - } - if (workItem == null || !workItem.isPresent() || workItem.get() == null) { + Optional workItem = + computation != null + ? workUnitClient.getStreamingConfigWorkItem(computation) + : workUnitClient.getGlobalStreamingConfigWorkItem(); + + if (!workItem.isPresent()) { return; } StreamingConfigTask config = workItem.get().getStreamingConfigTask(); @@ -1837,7 +1511,8 @@ private void getConfigFromDataflowService(@Nullable String computation) throws I addComputation( computationConfig.getComputationId(), mapTask, - computationConfig.getTransformUserNameToStateFamily()); + Optional.ofNullable(computationConfig.getTransformUserNameToStateFamily()) + .orElseGet(ImmutableMap::of)); } } @@ -2173,277 +1848,6 @@ private void invalidateStuckCommits() { } } - /** - * Class representing the state of a computation. - * - *

This class is synchronized, but only used from the dispatch and commit threads, so should - * not be heavily contended. Still, blocking work should not be done by it. - */ - static class ComputationState implements AutoCloseable { - - private final String computationId; - private final MapTask mapTask; - private final ImmutableMap transformUserNameToStateFamily; - // Map from key to work for the key. The first item in the queue is - // actively processing. Synchronized by itself. - private final Map> activeWork = new HashMap<>(); - private final BoundedQueueExecutor executor; - private final ConcurrentLinkedQueue executionStateQueue = - new ConcurrentLinkedQueue<>(); - private final WindmillStateCache.ForComputation computationStateCache; - - public ComputationState( - String computationId, - MapTask mapTask, - BoundedQueueExecutor executor, - Map transformUserNameToStateFamily, - WindmillStateCache.ForComputation computationStateCache) { - this.computationId = computationId; - this.mapTask = mapTask; - this.executor = executor; - this.transformUserNameToStateFamily = - transformUserNameToStateFamily != null - ? ImmutableMap.copyOf(transformUserNameToStateFamily) - : ImmutableMap.of(); - this.computationStateCache = computationStateCache; - Preconditions.checkNotNull(mapTask.getStageName()); - Preconditions.checkNotNull(mapTask.getSystemName()); - } - - public String getComputationId() { - return computationId; - } - - public MapTask getMapTask() { - return mapTask; - } - - public ImmutableMap getTransformUserNameToStateFamily() { - return transformUserNameToStateFamily; - } - - public ConcurrentLinkedQueue getExecutionStateQueue() { - return executionStateQueue; - } - - /** Mark the given shardedKey and work as active. */ - public boolean activateWork(ShardedKey shardedKey, Work work) { - synchronized (activeWork) { - Deque queue = activeWork.get(shardedKey); - if (queue != null) { - Preconditions.checkState(!queue.isEmpty()); - // Ensure we don't already have this work token queueud. - for (Work queuedWork : queue) { - if (queuedWork.getWorkItem().getWorkToken() == work.getWorkItem().getWorkToken()) { - return false; - } - } - // Queue the work for later processing. - queue.addLast(work); - return true; - } else { - queue = new ArrayDeque<>(); - queue.addLast(work); - activeWork.put(shardedKey, queue); - // Fall through to execute without the lock held. - } - } - executor.execute(work, work.getWorkItem().getSerializedSize()); - return true; - } - - /** - * Marks the work for the given shardedKey as complete. Schedules queued work for the key if - * any. - */ - public void completeWork(ShardedKey shardedKey, long workToken) { - Work nextWork; - synchronized (activeWork) { - Queue queue = activeWork.get(shardedKey); - if (queue == null) { - // Work may have been completed due to clearing of stuck commits. - LOG.warn( - "Unable to complete inactive work for key {} and token {}.", shardedKey, workToken); - return; - } - Work completedWork = queue.peek(); - // avoid Preconditions.checkState here to prevent eagerly evaluating the - // format string parameters for the error message. - if (completedWork == null) { - throw new IllegalStateException( - String.format( - "Active key %s without work, expected token %d", shardedKey, workToken)); - } - if (completedWork.getWorkItem().getWorkToken() != workToken) { - // Work may have been completed due to clearing of stuck commits. - LOG.warn( - "Unable to complete due to token mismatch for key {} and token {}, actual token was {}.", - shardedKey, - workToken, - completedWork.getWorkItem().getWorkToken()); - return; - } - queue.remove(); // We consumed the matching work item. - nextWork = queue.peek(); - if (nextWork == null) { - Preconditions.checkState(queue == activeWork.remove(shardedKey)); - } - } - if (nextWork != null) { - executor.forceExecute(nextWork, nextWork.getWorkItem().getSerializedSize()); - } - } - - public void invalidateStuckCommits(Instant stuckCommitDeadline) { - synchronized (activeWork) { - // Determine the stuck commit keys but complete them outside of iterating over - // activeWork as completeWork may delete the entry from activeWork. - Map stuckCommits = new HashMap<>(); - for (Map.Entry> entry : activeWork.entrySet()) { - ShardedKey shardedKey = entry.getKey(); - Work work = entry.getValue().peek(); - if (work.getState() == State.COMMITTING - && work.getStateStartTime().isBefore(stuckCommitDeadline)) { - LOG.error( - "Detected key {} stuck in COMMITTING state since {}, completing it with error.", - shardedKey, - work.getStateStartTime()); - stuckCommits.put(shardedKey, work.getWorkItem().getWorkToken()); - } - } - for (Map.Entry stuckCommit : stuckCommits.entrySet()) { - computationStateCache.invalidate( - stuckCommit.getKey().key(), stuckCommit.getKey().shardingKey()); - completeWork(stuckCommit.getKey(), stuckCommit.getValue()); - } - } - } - - /** Adds any work started before the refreshDeadline to the GetDataRequest builder. */ - public List getKeysToRefresh(Instant refreshDeadline) { - List result = new ArrayList<>(); - synchronized (activeWork) { - for (Map.Entry> entry : activeWork.entrySet()) { - ShardedKey shardedKey = entry.getKey(); - for (Work work : entry.getValue()) { - if (work.getStartTime().isBefore(refreshDeadline)) { - result.add( - Windmill.KeyedGetDataRequest.newBuilder() - .setKey(shardedKey.key()) - .setShardingKey(shardedKey.shardingKey()) - .setWorkToken(work.getWorkItem().getWorkToken()) - .addAllLatencyAttribution(work.getLatencyAttributions()) - .build()); - } - } - } - } - return result; - } - - private String elapsedString(Instant start, Instant end) { - Duration activeFor = new Duration(start, end); - // Duration's toString always starts with "PT"; remove that here. - return activeFor.toString().substring(2); - } - - public void printActiveWork(PrintWriter writer) { - final Instant now = Instant.now(); - // The max number of keys in COMMITTING or COMMIT_QUEUED status to be shown. - final int maxCommitPending = 50; - int commitPendingCount = 0; - writer.println( - ""); - writer.println( - ""); - // We use a StringBuilder in the synchronized section to buffer writes since the provided - // PrintWriter may block when flushing. - StringBuilder builder = new StringBuilder(); - synchronized (activeWork) { - for (Map.Entry> entry : activeWork.entrySet()) { - Queue queue = entry.getValue(); - Preconditions.checkNotNull(queue); - Work work = queue.peek(); - Preconditions.checkNotNull(work); - Windmill.WorkItem workItem = work.getWorkItem(); - State state = work.getState(); - if (state == State.COMMITTING || state == State.COMMIT_QUEUED) { - if (++commitPendingCount >= maxCommitPending) { - continue; - } - } - builder.append(""); - builder.append("\n"); - } - } - writer.print(builder.toString()); - writer.println("
KeyTokenQueuedActive ForStateState Active For
"); - builder.append(String.format("%016x", workItem.getShardingKey())); - builder.append(""); - builder.append(String.format("%016x", workItem.getWorkToken())); - builder.append(""); - builder.append(queue.size() - 1); - builder.append(""); - builder.append(elapsedString(work.getStartTime(), now)); - builder.append(""); - builder.append(state); - builder.append(""); - builder.append(elapsedString(work.getStateStartTime(), now)); - builder.append("
"); - if (commitPendingCount >= maxCommitPending) { - writer.println("
"); - writer.print("Skipped keys in COMMITTING/COMMIT_QUEUED: "); - writer.println(commitPendingCount - maxCommitPending); - writer.println("
"); - } - } - - @Override - public void close() throws Exception { - ExecutionState executionState; - while ((executionState = executionStateQueue.poll()) != null) { - executionState.getWorkExecutor().close(); - } - executionStateQueue.clear(); - } - } - - private static class ExecutionState { - - public final DataflowWorkExecutor workExecutor; - public final StreamingModeExecutionContext context; - public final @Nullable Coder keyCoder; - private final ExecutionStateTracker executionStateTracker; - - public ExecutionState( - DataflowWorkExecutor workExecutor, - StreamingModeExecutionContext context, - Coder keyCoder, - ExecutionStateTracker executionStateTracker) { - this.workExecutor = workExecutor; - this.context = context; - this.keyCoder = keyCoder; - this.executionStateTracker = executionStateTracker; - } - - public DataflowWorkExecutor getWorkExecutor() { - return workExecutor; - } - - public StreamingModeExecutionContext getContext() { - return context; - } - - public ExecutionStateTracker getExecutionStateTracker() { - return executionStateTracker; - } - - public @Nullable Coder getKeyCoder() { - return keyCoder; - } - } - private class HarnessDataProvider implements StatusDataProvider { @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java index c3c11716e4c7..82fbcd82c131 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java @@ -21,7 +21,7 @@ import com.google.api.services.dataflow.model.WorkItemServiceState; import com.google.api.services.dataflow.model.WorkItemStatus; import java.io.IOException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; +import java.util.Optional; /** Abstract base class describing a client for WorkItem work units. */ interface WorkUnitClient { @@ -31,14 +31,14 @@ interface WorkUnitClient { Optional getWorkItem() throws IOException; /** - * Returns a new global streaming config WorkItem, or returns {@link Optional#absent()} if no work + * Returns a new global streaming config WorkItem, or returns {@link Optional#empty()} if no work * was found. */ Optional getGlobalStreamingConfigWorkItem() throws IOException; /** * Returns a streaming config WorkItem for the given computation, or returns {@link - * Optional#absent()} if no work was found. + * Optional#empty()} if no work was found. */ Optional getStreamingConfigWorkItem(String computationId) throws IOException; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/NameContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/NameContext.java index 6188386a4e67..4f4a1c3834e5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/NameContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/NameContext.java @@ -31,7 +31,10 @@ public abstract class NameContext { * systemName} and a {@code userName}. */ public static NameContext create( - String stageName, String originalName, String systemName, String userName) { + String stageName, + @Nullable String originalName, + String systemName, + @Nullable String userName) { return new AutoValue_NameContext(stageName, originalName, systemName, userName); } @@ -44,7 +47,7 @@ public static NameContext forStage(String stageName) { } /** Returns the name of the stage this instruction is executing in. */ - public abstract @Nullable String stageName(); + public abstract String stageName(); /** * Returns the "original" name of this instruction. This name is a short name assigned by the SDK diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java new file mode 100644 index 000000000000..529bb0a41907 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; + +import java.io.PrintWriter; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Queue; +import java.util.function.BiConsumer; +import java.util.stream.Stream; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.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; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages the active {@link Work} queues for their {@link ShardedKey}(s). Gives an interface to + * activate, queue, and complete {@link Work} (including invalidating stuck {@link Work}). + */ +@ThreadSafe +final class ActiveWorkState { + private static final Logger LOG = LoggerFactory.getLogger(ActiveWorkState.class); + + /* The max number of keys in COMMITTING or COMMIT_QUEUED status to be shown.*/ + private static final int MAX_PRINTABLE_COMMIT_PENDING_KEYS = 50; + + /** + * Map from {@link ShardedKey} to {@link Work} for the key. The first item in the {@link + * Queue} is actively processing. + */ + @GuardedBy("this") + private final Map> activeWork; + + @GuardedBy("this") + private final WindmillStateCache.ForComputation computationStateCache; + + private ActiveWorkState( + Map> activeWork, + WindmillStateCache.ForComputation computationStateCache) { + this.activeWork = activeWork; + this.computationStateCache = computationStateCache; + } + + static ActiveWorkState create(WindmillStateCache.ForComputation computationStateCache) { + return new ActiveWorkState(new HashMap<>(), computationStateCache); + } + + @VisibleForTesting + static ActiveWorkState forTesting( + Map> activeWork, + WindmillStateCache.ForComputation computationStateCache) { + return new ActiveWorkState(activeWork, computationStateCache); + } + + /** + * Activates {@link Work} for the {@link ShardedKey}. Outcome can be 1 of 3 {@link + * ActivateWorkResult} + * + *

1. EXECUTE: The {@link ShardedKey} has not been seen before, create a {@link Queue} + * for the key. The caller should execute the work. + * + *

2. DUPLICATE: A work queue for the {@link ShardedKey} exists, and the work already exists in + * the {@link ShardedKey}'s work queue, mark the {@link Work} as a duplicate. + * + *

3. QUEUED: A work queue for the {@link ShardedKey} exists, and the work is not in the key's + * work queue, queue the work for later processing. + */ + synchronized ActivateWorkResult activateWorkForKey(ShardedKey shardedKey, Work work) { + Deque workQueue = activeWork.getOrDefault(shardedKey, new ArrayDeque<>()); + + // This key does not have any work queued up on it. Create one, insert Work, and mark the work + // to be executed. + if (!activeWork.containsKey(shardedKey) || workQueue.isEmpty()) { + workQueue.addLast(work); + activeWork.put(shardedKey, workQueue); + return ActivateWorkResult.EXECUTE; + } + + // Ensure we don't already have this work token queued. + for (Work queuedWork : workQueue) { + if (queuedWork.getWorkItem().getWorkToken() == work.getWorkItem().getWorkToken()) { + return ActivateWorkResult.DUPLICATE; + } + } + + // Queue the work for later processing. + workQueue.addLast(work); + return ActivateWorkResult.QUEUED; + } + + /** + * Removes the complete work from the {@link Queue}. The {@link Work} is marked as completed + * if its workToken matches the one that is passed in. Returns the next {@link Work} in the {@link + * ShardedKey}'s work queue, if one exists else removes the {@link ShardedKey} from {@link + * #activeWork}. + */ + synchronized Optional completeWorkAndGetNextWorkForKey( + ShardedKey shardedKey, long workToken) { + @Nullable Queue workQueue = activeWork.get(shardedKey); + if (workQueue == null) { + // Work may have been completed due to clearing of stuck commits. + LOG.warn("Unable to complete inactive work for key {} and token {}.", shardedKey, workToken); + return Optional.empty(); + } + removeCompletedWorkFromQueue(workQueue, shardedKey, workToken); + return getNextWork(workQueue, shardedKey); + } + + private synchronized void removeCompletedWorkFromQueue( + Queue workQueue, ShardedKey shardedKey, long workToken) { + // avoid Preconditions.checkState here to prevent eagerly evaluating the + // format string parameters for the error message. + Work completedWork = + Optional.ofNullable(workQueue.peek()) + .orElseThrow( + () -> + new IllegalStateException( + String.format( + "Active key %s without work, expected token %d", + shardedKey, workToken))); + + if (completedWork.getWorkItem().getWorkToken() != workToken) { + // Work may have been completed due to clearing of stuck commits. + LOG.warn( + "Unable to complete due to token mismatch for key {} and token {}, actual token was {}.", + shardedKey, + workToken, + completedWork.getWorkItem().getWorkToken()); + return; + } + + // We consumed the matching work item. + workQueue.remove(); + } + + private synchronized Optional getNextWork(Queue workQueue, ShardedKey shardedKey) { + Optional nextWork = Optional.ofNullable(workQueue.peek()); + if (!nextWork.isPresent()) { + Preconditions.checkState(workQueue == activeWork.remove(shardedKey)); + } + + return nextWork; + } + + /** + * Invalidates all {@link Work} that is in the {@link Work.State#COMMITTING} state which started + * before the stuckCommitDeadline. + */ + synchronized void invalidateStuckCommits( + Instant stuckCommitDeadline, BiConsumer shardedKeyAndWorkTokenConsumer) { + for (Entry shardedKeyAndWorkToken : + getStuckCommitsAt(stuckCommitDeadline).entrySet()) { + ShardedKey shardedKey = shardedKeyAndWorkToken.getKey(); + long workToken = shardedKeyAndWorkToken.getValue(); + computationStateCache.invalidate(shardedKey.key(), shardedKey.shardingKey()); + shardedKeyAndWorkTokenConsumer.accept(shardedKey, workToken); + } + } + + private synchronized ImmutableMap getStuckCommitsAt( + Instant stuckCommitDeadline) { + // Determine the stuck commit keys but complete them outside the loop iterating over + // activeWork as completeWork may delete the entry from activeWork. + ImmutableMap.Builder stuckCommits = ImmutableMap.builder(); + for (Entry> entry : activeWork.entrySet()) { + ShardedKey shardedKey = entry.getKey(); + @Nullable Work work = entry.getValue().peek(); + if (work != null) { + if (work.isStuckCommittingAt(stuckCommitDeadline)) { + LOG.error( + "Detected key {} stuck in COMMITTING state since {}, completing it with error.", + shardedKey, + work.getStateStartTime()); + stuckCommits.put(shardedKey, work.getWorkItem().getWorkToken()); + } + } + } + + return stuckCommits.build(); + } + + synchronized ImmutableList getKeysToRefresh(Instant refreshDeadline) { + return activeWork.entrySet().stream() + .flatMap(entry -> toKeyedGetDataRequestStream(entry, refreshDeadline)) + .collect(toImmutableList()); + } + + private static Stream toKeyedGetDataRequestStream( + Entry> shardedKeyAndWorkQueue, Instant refreshDeadline) { + ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); + Deque workQueue = shardedKeyAndWorkQueue.getValue(); + + return workQueue.stream() + .filter(work -> work.getStartTime().isBefore(refreshDeadline)) + .map( + work -> + Windmill.KeyedGetDataRequest.newBuilder() + .setKey(shardedKey.key()) + .setShardingKey(shardedKey.shardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .addAllLatencyAttribution(work.getLatencyAttributions()) + .build()); + } + + synchronized void printActiveWork(PrintWriter writer, Instant now) { + writer.println( + ""); + writer.println( + ""); + // Use StringBuilder because we are appending in loop. + StringBuilder activeWorkStatus = new StringBuilder(); + int commitsPendingCount = 0; + for (Map.Entry> entry : activeWork.entrySet()) { + Queue workQueue = Preconditions.checkNotNull(entry.getValue()); + Work activeWork = Preconditions.checkNotNull(workQueue.peek()); + Windmill.WorkItem workItem = activeWork.getWorkItem(); + if (activeWork.isCommitPending()) { + if (++commitsPendingCount >= MAX_PRINTABLE_COMMIT_PENDING_KEYS) { + continue; + } + } + activeWorkStatus.append(""); + activeWorkStatus.append("\n"); + } + + writer.print(activeWorkStatus); + writer.println("
KeyTokenQueuedActive ForStateState Active For
"); + activeWorkStatus.append(String.format("%016x", workItem.getShardingKey())); + activeWorkStatus.append(""); + activeWorkStatus.append(String.format("%016x", workItem.getWorkToken())); + activeWorkStatus.append(""); + activeWorkStatus.append(workQueue.size() - 1); + activeWorkStatus.append(""); + activeWorkStatus.append(elapsedString(activeWork.getStartTime(), now)); + activeWorkStatus.append(""); + activeWorkStatus.append(activeWork.getState()); + activeWorkStatus.append(""); + activeWorkStatus.append(elapsedString(activeWork.getStateStartTime(), now)); + activeWorkStatus.append("
"); + + if (commitsPendingCount >= MAX_PRINTABLE_COMMIT_PENDING_KEYS) { + writer.println("
"); + writer.print("Skipped keys in COMMITTING/COMMIT_QUEUED: "); + writer.println(commitsPendingCount - MAX_PRINTABLE_COMMIT_PENDING_KEYS); + writer.println("
"); + } + } + + private static String elapsedString(Instant start, Instant end) { + Duration activeFor = new Duration(start, end); + // Duration's toString always starts with "PT"; remove that here. + return activeFor.toString().substring(2); + } + + enum ActivateWorkResult { + QUEUED, + EXECUTE, + DUPLICATE + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Commit.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Commit.java new file mode 100644 index 000000000000..946897967561 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Commit.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import com.google.auto.value.AutoValue; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; + +/** Value class for a queued commit. */ +@AutoValue +public abstract class Commit { + + public static Commit create( + WorkItemCommitRequest request, ComputationState computationState, Work work) { + Preconditions.checkArgument(request.getSerializedSize() > 0); + return new AutoValue_Commit(request, computationState, work); + } + + public abstract WorkItemCommitRequest request(); + + public abstract ComputationState computationState(); + + public abstract Work work(); + + public final int getSize() { + return request().getSerializedSize(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java new file mode 100644 index 000000000000..a902d2b13a77 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import com.google.api.services.dataflow.model.MapTask; +import java.io.PrintWriter; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Instant; + +/** + * Class representing the state of a computation. + * + *

This class is synchronized, but only used from the dispatch and commit threads, so should not + * be heavily contended. Still, blocking work should not be done by it. + */ +public class ComputationState implements AutoCloseable { + private final String computationId; + private final MapTask mapTask; + private final ImmutableMap transformUserNameToStateFamily; + private final ActiveWorkState activeWorkState; + private final BoundedQueueExecutor executor; + private final ConcurrentLinkedQueue executionStateQueue; + + public ComputationState( + String computationId, + MapTask mapTask, + BoundedQueueExecutor executor, + Map transformUserNameToStateFamily, + WindmillStateCache.ForComputation computationStateCache) { + Preconditions.checkNotNull(mapTask.getStageName()); + Preconditions.checkNotNull(mapTask.getSystemName()); + this.computationId = computationId; + this.mapTask = mapTask; + this.executor = executor; + this.transformUserNameToStateFamily = ImmutableMap.copyOf(transformUserNameToStateFamily); + this.executionStateQueue = new ConcurrentLinkedQueue<>(); + this.activeWorkState = ActiveWorkState.create(computationStateCache); + } + + public String getComputationId() { + return computationId; + } + + public MapTask getMapTask() { + return mapTask; + } + + public ImmutableMap getTransformUserNameToStateFamily() { + return transformUserNameToStateFamily; + } + + public ConcurrentLinkedQueue getExecutionStateQueue() { + return executionStateQueue; + } + + /** + * Mark the given {@link ShardedKey} and {@link Work} as active, and schedules execution of {@link + * Work} if there is no active {@link Work} for the {@link ShardedKey} already processing. + */ + public boolean activateWork(ShardedKey shardedKey, Work work) { + switch (activeWorkState.activateWorkForKey(shardedKey, work)) { + case DUPLICATE: + return false; + case QUEUED: + return true; + case EXECUTE: + { + execute(work); + return true; + } + default: + // This will never happen, the switch is exhaustive. + throw new IllegalStateException("Unrecognized ActivateWorkResult"); + } + } + + /** + * Marks the work for the given shardedKey as complete. Schedules queued work for the key if any. + */ + public void completeWorkAndScheduleNextWorkForKey(ShardedKey shardedKey, long workToken) { + activeWorkState + .completeWorkAndGetNextWorkForKey(shardedKey, workToken) + .ifPresent(this::forceExecute); + } + + public void invalidateStuckCommits(Instant stuckCommitDeadline) { + activeWorkState.invalidateStuckCommits( + stuckCommitDeadline, this::completeWorkAndScheduleNextWorkForKey); + } + + private void execute(Work work) { + executor.execute(work, work.getWorkItem().getSerializedSize()); + } + + private void forceExecute(Work work) { + executor.forceExecute(work, work.getWorkItem().getSerializedSize()); + } + + /** Adds any work started before the refreshDeadline to the GetDataRequest builder. */ + public List getKeysToRefresh(Instant refreshDeadline) { + return activeWorkState.getKeysToRefresh(refreshDeadline); + } + + public void printActiveWork(PrintWriter writer) { + activeWorkState.printActiveWork(writer, Instant.now()); + } + + @Override + public void close() throws Exception { + @Nullable ExecutionState executionState; + while ((executionState = executionStateQueue.poll()) != null) { + executionState.workExecutor().close(); + } + executionStateQueue.clear(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java new file mode 100644 index 000000000000..ba35179a75b3 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import org.apache.beam.runners.core.metrics.ExecutionStateTracker; +import org.apache.beam.runners.dataflow.worker.DataflowWorkExecutor; +import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext; +import org.apache.beam.sdk.coders.Coder; + +@AutoValue +public abstract class ExecutionState { + + public abstract DataflowWorkExecutor workExecutor(); + + public abstract StreamingModeExecutionContext context(); + + public abstract Optional> keyCoder(); + + public abstract ExecutionStateTracker executionStateTracker(); + + public static ExecutionState.Builder builder() { + return new AutoValue_ExecutionState.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setWorkExecutor(DataflowWorkExecutor workExecutor); + + public abstract Builder setContext(StreamingModeExecutionContext context); + + public abstract Builder setKeyCoder(Coder keyCoder); + + public abstract Builder setExecutionStateTracker(ExecutionStateTracker executionStateTracker); + + public abstract ExecutionState build(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/KeyCommitTooLargeException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/KeyCommitTooLargeException.java new file mode 100644 index 000000000000..090d9981309e --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/KeyCommitTooLargeException.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; + +public final class KeyCommitTooLargeException extends Exception { + + public static KeyCommitTooLargeException causedBy( + String computationId, long byteLimit, Windmill.WorkItemCommitRequest request) { + StringBuilder message = new StringBuilder(); + message.append("Commit request for stage "); + message.append(computationId); + message.append(" and key "); + message.append(request.getKey().toStringUtf8()); + if (request.getSerializedSize() > 0) { + message.append( + " has size " + + request.getSerializedSize() + + " which is more than the limit of " + + byteLimit); + } else { + message.append(" is larger than 2GB and cannot be processed"); + } + message.append( + ". This may be caused by grouping a very " + + "large amount of data in a single window without using Combine," + + " or by producing a large amount of data from a single input element."); + return new KeyCommitTooLargeException(message.toString()); + } + + private KeyCommitTooLargeException(String message) { + super(message); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java new file mode 100644 index 000000000000..86433d9e6752 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import com.google.auto.value.AutoValue; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; + +@AutoValue +public abstract class ShardedKey { + + public static ShardedKey create(ByteString key, long shardingKey) { + return new AutoValue_ShardedKey(key, shardingKey); + } + + public abstract ByteString key(); + + public abstract long shardingKey(); + + @Override + public final String toString() { + return String.format("%016x", shardingKey()); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java new file mode 100644 index 000000000000..b514dfc84bb9 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import static org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.THROTTLING_MSECS_METRIC_NAME; + +import com.google.api.services.dataflow.model.CounterStructuredName; +import com.google.api.services.dataflow.model.CounterUpdate; +import com.google.auto.value.AutoValue; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics; +import org.apache.beam.runners.dataflow.worker.MetricsContainerRegistry; +import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker; +import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionStateRegistry; +import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer; +import org.apache.beam.runners.dataflow.worker.counters.Counter; +import org.apache.beam.runners.dataflow.worker.counters.CounterSet; +import org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor; +import org.apache.beam.runners.dataflow.worker.counters.NameContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; + +/** Contains a few of the stage specific fields. E.g. metrics container registry, counters etc. */ +@AutoValue +public abstract class StageInfo { + public static StageInfo create( + String stageName, String systemName, StreamingDataflowWorker worker) { + NameContext nameContext = NameContext.create(stageName, null, systemName, null); + CounterSet deltaCounters = new CounterSet(); + return new AutoValue_StageInfo( + stageName, + systemName, + StreamingStepMetricsContainer.createRegistry(), + new StreamingModeExecutionStateRegistry(worker), + deltaCounters, + deltaCounters.longSum( + DataflowSystemMetrics.StreamingPerStageSystemCounterNames.THROTTLED_MSECS.counterName( + nameContext)), + deltaCounters.longSum( + DataflowSystemMetrics.StreamingPerStageSystemCounterNames.TOTAL_PROCESSING_MSECS + .counterName(nameContext)), + deltaCounters.longSum( + DataflowSystemMetrics.StreamingPerStageSystemCounterNames.TIMER_PROCESSING_MSECS + .counterName(nameContext))); + } + + public abstract String stageName(); + + public abstract String systemName(); + + public abstract MetricsContainerRegistry + metricsContainerRegistry(); + + public abstract StreamingModeExecutionStateRegistry executionStateRegistry(); + + public abstract CounterSet deltaCounters(); + + public abstract Counter throttledMsecs(); + + public abstract Counter totalProcessingMsecs(); + + public abstract Counter timerProcessingMsecs(); + + public List extractCounterUpdates() { + List counterUpdates = new ArrayList<>(); + Iterables.addAll( + counterUpdates, + StreamingStepMetricsContainer.extractMetricUpdates(metricsContainerRegistry())); + Iterables.addAll(counterUpdates, executionStateRegistry().extractUpdates(false)); + for (CounterUpdate counterUpdate : counterUpdates) { + translateKnownStepCounters(counterUpdate); + } + counterUpdates.addAll( + deltaCounters().extractModifiedDeltaUpdates(DataflowCounterUpdateExtractor.INSTANCE)); + return counterUpdates; + } + + /** + * Checks if the step counter affects any per-stage counters. Currently 'throttled_millis' is the + * only counter updated. + */ + private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) { + CounterStructuredName structuredName = + stepCounterUpdate.getStructuredNameAndMetadata().getName(); + if ((THROTTLING_MSECS_METRIC_NAME.getNamespace().equals(structuredName.getOriginNamespace()) + && THROTTLING_MSECS_METRIC_NAME.getName().equals(structuredName.getName())) + || (StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME + .getNamespace() + .equals(structuredName.getOriginNamespace()) + && StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME + .getName() + .equals(structuredName.getName()))) { + long msecs = DataflowCounterUpdateExtractor.splitIntToLong(stepCounterUpdate.getInteger()); + if (msecs > 0) { + throttledMsecs().addValue(msecs); + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WeightedBoundedQueue.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WeightedBoundedQueue.java new file mode 100644 index 000000000000..f2893f3e7191 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WeightedBoundedQueue.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Bounded set of queues, with a maximum total weight. */ +public final class WeightedBoundedQueue { + + private final LinkedBlockingQueue queue; + private final int maxWeight; + private final Semaphore limit; + private final Function weigher; + + private WeightedBoundedQueue( + LinkedBlockingQueue linkedBlockingQueue, + int maxWeight, + Semaphore limit, + Function weigher) { + this.queue = linkedBlockingQueue; + this.maxWeight = maxWeight; + this.limit = limit; + this.weigher = weigher; + } + + public static WeightedBoundedQueue create(int maxWeight, Function weigherFn) { + return new WeightedBoundedQueue<>( + new LinkedBlockingQueue<>(), maxWeight, new Semaphore(maxWeight, true), weigherFn); + } + + /** + * Adds the value to the queue, blocking if this would cause the overall weight to exceed the + * limit. + */ + public void put(V value) { + limit.acquireUninterruptibly(weigher.apply(value)); + queue.add(value); + } + + /** Returns and removes the next value, or null if there is no such value. */ + public @Nullable V poll() { + V result = queue.poll(); + if (result != null) { + limit.release(weigher.apply(result)); + } + return result; + } + + /** + * Retrieves and removes the head of this queue, waiting up to the specified wait time if + * necessary for an element to become available. + * + * @param timeout how long to wait before giving up, in units of {@code unit} + * @param unit a {@code TimeUnit} determining how to interpret the {@code timeout} parameter + * @return the head of this queue, or {@code null} if the specified waiting time elapses before an + * element is available + * @throws InterruptedException if interrupted while waiting + */ + public @Nullable V poll(long timeout, TimeUnit unit) throws InterruptedException { + V result = queue.poll(timeout, unit); + if (result != null) { + limit.release(weigher.apply(result)); + } + return result; + } + + /** Returns and removes the next value, or blocks until one is available. */ + public @Nullable V take() throws InterruptedException { + V result = queue.take(); + limit.release(weigher.apply(result)); + return result; + } + + /** Returns the current weight of the queue. */ + public int queuedElementsWeight() { + return maxWeight - limit.availablePermits(); + } + + public int size() { + return queue.size(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java new file mode 100644 index 000000000000..cc3f6d1b12b2 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import com.google.auto.value.AutoValue; +import java.util.ArrayList; +import java.util.Collection; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.Supplier; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.joda.time.Duration; +import org.joda.time.Instant; + +@NotThreadSafe +public class Work implements Runnable { + + private final Windmill.WorkItem workItem; + private final Supplier clock; + private final Instant startTime; + private final Map totalDurationPerState; + private final Consumer processWorkFn; + private TimedState currentState; + + private Work(Windmill.WorkItem workItem, Supplier clock, Consumer processWorkFn) { + this.workItem = workItem; + this.clock = clock; + this.processWorkFn = processWorkFn; + this.startTime = clock.get(); + this.totalDurationPerState = new EnumMap<>(Windmill.LatencyAttribution.State.class); + this.currentState = TimedState.initialState(startTime); + } + + public static Work create( + Windmill.WorkItem workItem, + Supplier clock, + Collection getWorkStreamLatencies, + Consumer processWorkFn) { + Work work = new Work(workItem, clock, processWorkFn); + work.recordGetWorkStreamLatencies(getWorkStreamLatencies); + return work; + } + + @Override + public void run() { + processWorkFn.accept(this); + } + + public Windmill.WorkItem getWorkItem() { + return workItem; + } + + public Instant getStartTime() { + return startTime; + } + + public State getState() { + return currentState.state(); + } + + public void setState(State state) { + Instant now = clock.get(); + totalDurationPerState.compute( + this.currentState.state().toLatencyAttributionState(), + (s, d) -> + new Duration(this.currentState.startTime(), now).plus(d == null ? Duration.ZERO : d)); + this.currentState = TimedState.create(state, now); + } + + public boolean isCommitPending() { + return currentState.isCommitPending(); + } + + public Instant getStateStartTime() { + return currentState.startTime(); + } + + private void recordGetWorkStreamLatencies( + Collection getWorkStreamLatencies) { + for (Windmill.LatencyAttribution latency : getWorkStreamLatencies) { + totalDurationPerState.put( + latency.getState(), Duration.millis(latency.getTotalDurationMillis())); + } + } + + public Collection getLatencyAttributions() { + List list = new ArrayList<>(); + for (Windmill.LatencyAttribution.State state : Windmill.LatencyAttribution.State.values()) { + Duration duration = totalDurationPerState.getOrDefault(state, Duration.ZERO); + if (state == this.currentState.state().toLatencyAttributionState()) { + duration = duration.plus(new Duration(this.currentState.startTime(), clock.get())); + } + if (duration.equals(Duration.ZERO)) { + continue; + } + list.add( + Windmill.LatencyAttribution.newBuilder() + .setState(state) + .setTotalDurationMillis(duration.getMillis()) + .build()); + } + return list; + } + + boolean isStuckCommittingAt(Instant stuckCommitDeadline) { + return currentState.state() == Work.State.COMMITTING + && currentState.startTime().isBefore(stuckCommitDeadline); + } + + public enum State { + QUEUED(Windmill.LatencyAttribution.State.QUEUED), + PROCESSING(Windmill.LatencyAttribution.State.ACTIVE), + READING(Windmill.LatencyAttribution.State.READING), + COMMIT_QUEUED(Windmill.LatencyAttribution.State.COMMITTING), + COMMITTING(Windmill.LatencyAttribution.State.COMMITTING), + GET_WORK_IN_WINDMILL_WORKER(Windmill.LatencyAttribution.State.GET_WORK_IN_WINDMILL_WORKER), + GET_WORK_IN_TRANSIT_TO_DISPATCHER( + Windmill.LatencyAttribution.State.GET_WORK_IN_TRANSIT_TO_DISPATCHER), + GET_WORK_IN_TRANSIT_TO_USER_WORKER( + Windmill.LatencyAttribution.State.GET_WORK_IN_TRANSIT_TO_USER_WORKER); + + private final Windmill.LatencyAttribution.State latencyAttributionState; + + State(Windmill.LatencyAttribution.State latencyAttributionState) { + this.latencyAttributionState = latencyAttributionState; + } + + Windmill.LatencyAttribution.State toLatencyAttributionState() { + return latencyAttributionState; + } + } + + /** + * Represents the current state of an instance of {@link Work}. Contains the {@link State} and + * {@link Instant} when it started. + */ + @AutoValue + abstract static class TimedState { + private static TimedState create(State state, Instant startTime) { + return new AutoValue_Work_TimedState(state, startTime); + } + + private static TimedState initialState(Instant startTime) { + return create(State.QUEUED, startTime); + } + + private boolean isCommitPending() { + return state() == Work.State.COMMITTING || state() == Work.State.COMMIT_QUEUED; + } + + abstract State state(); + + abstract Instant startTime(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java index 8d5660548e75..b4f544129db6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java @@ -37,11 +37,11 @@ import com.google.api.services.dataflow.model.WorkItemStatus; import java.io.IOException; import java.util.ArrayList; +import java.util.Optional; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.util.TimeUtil; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.FastNanoClockAndSleeper; -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; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; @@ -61,16 +61,10 @@ @RunWith(JUnit4.class) public class BatchDataflowWorkerTest { - private static class WorkerException extends Exception {} - @Rule public FastNanoClockAndSleeper clockAndSleeper = new FastNanoClockAndSleeper(); - @Mock WorkUnitClient mockWorkUnitClient; - @Mock DataflowWorkProgressUpdater mockProgressUpdater; - @Mock DataflowWorkExecutor mockWorkExecutor; - DataflowWorkerHarnessOptions options; @Before @@ -98,7 +92,7 @@ public void testWhenNoWorkIsReturnedThatWeImmediatelyRetry() throws Exception { workItem.setReportStatusInterval(TimeUtil.toCloudDuration(Duration.standardMinutes(1))); when(mockWorkUnitClient.getWorkItem()) - .thenReturn(Optional.absent()) + .thenReturn(Optional.empty()) .thenReturn(Optional.of(workItem)); assertTrue(worker.getAndPerformWork()); @@ -138,7 +132,7 @@ public void testWhenProcessingWorkUnitFailsWeReportStatus() throws Exception { Throwable error = errorCaptor.getValue(); assertThat(error, notNullValue()); - assertThat(error.getMessage(), equalTo("Unknown kind of work item: " + workItem.toString())); + assertThat(error.getMessage(), equalTo("Unknown kind of work item: " + workItem)); } @Test @@ -168,8 +162,9 @@ public void testStopProgressReportInCaseOfFailure() throws Exception { @Test public void testIsSplitResponseTooLarge() throws IOException { SourceSplitResponse splitResponse = new SourceSplitResponse(); - splitResponse.setShards( - ImmutableList.of(new SourceSplitShard(), new SourceSplitShard())); + splitResponse.setShards(ImmutableList.of(new SourceSplitShard(), new SourceSplitShard())); assertThat(DataflowApiUtils.computeSerializedSizeBytes(splitResponse), greaterThan(0L)); } + + private static class WorkerException extends Exception {} } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java index e8b5ce8d0df2..3c63f3cc19d2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java @@ -34,6 +34,7 @@ import com.google.api.services.dataflow.model.SeqMapTask; import com.google.api.services.dataflow.model.WorkItem; import java.io.IOException; +import java.util.Optional; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; @@ -42,7 +43,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.FastNanoClockAndSleeper; -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; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.junit.Before; @@ -61,6 +61,9 @@ @RunWith(JUnit4.class) public class DataflowWorkUnitClientTest { private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkUnitClientTest.class); + private static final String PROJECT_ID = "TEST_PROJECT_ID"; + private static final String JOB_ID = "TEST_JOB_ID"; + private static final String WORKER_ID = "TEST_WORKER_ID"; @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); @Rule public TestRule restoreLogging = new RestoreDataflowLoggingMDC(); @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -69,10 +72,6 @@ public class DataflowWorkUnitClientTest { @Mock private MockLowLevelHttpRequest request; private DataflowWorkerHarnessOptions pipelineOptions; - private static final String PROJECT_ID = "TEST_PROJECT_ID"; - private static final String JOB_ID = "TEST_JOB_ID"; - private static final String WORKER_ID = "TEST_WORKER_ID"; - @Before public void setUp() throws Exception { MockitoAnnotations.initMocks(this); @@ -104,10 +103,10 @@ public void testCloudServiceCall() throws Exception { .fromString(request.getContentAsString(), LeaseWorkItemRequest.class); assertEquals(WORKER_ID, actualRequest.getWorkerId()); assertEquals( - ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), + ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), actualRequest.getWorkerCapabilities()); assertEquals( - ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), actualRequest.getWorkItemTypes()); assertEquals("1234", DataflowWorkerLoggingMDC.getWorkId()); } @@ -151,17 +150,17 @@ public void testCloudServiceCallNoWorkPresent() throws Exception { WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); - assertEquals(Optional.absent(), client.getWorkItem()); + assertEquals(Optional.empty(), client.getWorkItem()); LeaseWorkItemRequest actualRequest = Transport.getJsonFactory() .fromString(request.getContentAsString(), LeaseWorkItemRequest.class); assertEquals(WORKER_ID, actualRequest.getWorkerId()); assertEquals( - ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), + ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), actualRequest.getWorkerCapabilities()); assertEquals( - ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), actualRequest.getWorkItemTypes()); } @@ -175,17 +174,17 @@ public void testCloudServiceCallNoWorkId() throws Exception { WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); - assertEquals(Optional.absent(), client.getWorkItem()); + assertEquals(Optional.empty(), client.getWorkItem()); LeaseWorkItemRequest actualRequest = Transport.getJsonFactory() .fromString(request.getContentAsString(), LeaseWorkItemRequest.class); assertEquals(WORKER_ID, actualRequest.getWorkerId()); assertEquals( - ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), + ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), actualRequest.getWorkerCapabilities()); assertEquals( - ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), actualRequest.getWorkItemTypes()); } @@ -195,17 +194,17 @@ public void testCloudServiceCallNoWorkItem() throws Exception { WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); - assertEquals(Optional.absent(), client.getWorkItem()); + assertEquals(Optional.empty(), client.getWorkItem()); LeaseWorkItemRequest actualRequest = Transport.getJsonFactory() .fromString(request.getContentAsString(), LeaseWorkItemRequest.class); assertEquals(WORKER_ID, actualRequest.getWorkerId()); assertEquals( - ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), + ImmutableList.of(WORKER_ID, "remote_source", "custom_source"), actualRequest.getWorkerCapabilities()); assertEquals( - ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), actualRequest.getWorkItemTypes()); } 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 95b3a43ebf49..82fc38055a88 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 @@ -28,6 +28,8 @@ import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.eq; @@ -65,6 +67,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.PriorityQueue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -92,8 +95,10 @@ import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.runners.dataflow.util.PropertyNames; import org.apache.beam.runners.dataflow.util.Structs; -import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.ShardedKey; import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; @@ -117,7 +122,6 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CollectionCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; @@ -158,7 +162,6 @@ import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; -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.cache.CacheStats; 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; @@ -191,36 +194,16 @@ @SuppressWarnings("unused") public class StreamingDataflowWorkerTest { - private final boolean streamingEngine; - - @Parameterized.Parameters(name = "{index}: [streamingEngine={0}]") - public static Iterable data() { - return Arrays.asList(new Object[][] {{false}, {true}}); - } - - public StreamingDataflowWorkerTest(Boolean streamingEngine) { - this.streamingEngine = streamingEngine; - } - private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorkerTest.class); - private static final IntervalWindow DEFAULT_WINDOW = new IntervalWindow(new Instant(1234), Duration.millis(1000)); - private static final IntervalWindow WINDOW_AT_ZERO = new IntervalWindow(new Instant(0), new Instant(1000)); - private static final IntervalWindow WINDOW_AT_ONE_SECOND = new IntervalWindow(new Instant(1000), new Instant(2000)); - private static final Coder DEFAULT_WINDOW_CODER = IntervalWindow.getCoder(); private static final Coder> DEFAULT_WINDOW_COLLECTION_CODER = CollectionCoder.of(DEFAULT_WINDOW_CODER); - - private byte[] intervalWindowBytes(IntervalWindow window) throws Exception { - return CoderUtils.encodeToByteArray(DEFAULT_WINDOW_COLLECTION_CODER, Arrays.asList(window)); - } - // Default values that are unimportant for correctness, but must be consistent // between pieces of this test suite private static final String DEFAULT_COMPUTATION_ID = "computation"; @@ -242,14 +225,26 @@ private byte[] intervalWindowBytes(IntervalWindow window) throws Exception { private static final ByteString DEFAULT_KEY_BYTES = ByteString.copyFromUtf8(DEFAULT_KEY_STRING); private static final String DEFAULT_DATA_STRING = "data"; private static final String DEFAULT_DESTINATION_STREAM_ID = "out"; - - @Rule public BlockingFn blockingFn = new BlockingFn(); - @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); - @Rule public ErrorCollector errorCollector = new ErrorCollector(); - - WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); - HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); - + private static final Function EMPTY_DATA_RESPONDER = + (GetDataRequest request) -> { + GetDataResponse.Builder builder = GetDataResponse.newBuilder(); + for (ComputationGetDataRequest compRequest : request.getRequestsList()) { + ComputationGetDataResponse.Builder compBuilder = + builder.addDataBuilder().setComputationId(compRequest.getComputationId()); + for (KeyedGetDataRequest keyRequest : compRequest.getRequestsList()) { + KeyedGetDataResponse.Builder keyBuilder = + compBuilder + .addDataBuilder() + .setKey(keyRequest.getKey()) + .setShardingKey(keyRequest.getShardingKey()); + keyBuilder.addAllValues(keyRequest.getValuesToFetchList()); + keyBuilder.addAllBags(keyRequest.getBagsToFetchList()); + keyBuilder.addAllWatermarkHolds(keyRequest.getWatermarkHoldsToFetchList()); + } + } + return builder.build(); + }; + private final boolean streamingEngine; private final Supplier idGenerator = new Supplier() { private final AtomicLong idGenerator = new AtomicLong(1L); @@ -259,6 +254,50 @@ public Long get() { return idGenerator.getAndIncrement(); } }; + @Rule public BlockingFn blockingFn = new BlockingFn(); + @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); + @Rule public ErrorCollector errorCollector = new ErrorCollector(); + WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); + HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); + + public StreamingDataflowWorkerTest(Boolean streamingEngine) { + this.streamingEngine = streamingEngine; + } + + @Parameterized.Parameters(name = "{index}: [streamingEngine={0}]") + public static Iterable data() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } + + private static CounterUpdate getCounter(Iterable counters, String name) { + for (CounterUpdate counter : counters) { + if (counter.getNameAndKind().getName().equals(name)) { + return counter; + } + } + return null; + } + + static Work createMockWork(long workToken) { + return Work.create( + Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), + Instant::now, + Collections.emptyList(), + work -> {}); + } + + static Work createMockWork(long workToken, Consumer processWorkFn) { + return Work.create( + Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), + Instant::now, + Collections.emptyList(), + work -> {}); + } + + private byte[] intervalWindowBytes(IntervalWindow window) throws Exception { + return CoderUtils.encodeToByteArray( + DEFAULT_WINDOW_COLLECTION_CODER, Collections.singletonList(window)); + } private String keyStringForIndex(int index) { return DEFAULT_KEY_STRING + index; @@ -273,7 +312,7 @@ private ParallelInstruction makeWindowingSourceInstruction(Coder coder) { CloudObject.forClassName( "com.google.cloud.dataflow.sdk.util.TimerOrElement$TimerOrElementCoder"); List component = - Collections.singletonList(CloudObjects.asCloudObject(coder, /*sdkComponents=*/ null)); + Collections.singletonList(CloudObjects.asCloudObject(coder, /* sdkComponents= */ null)); Structs.addList(timerCloudObject, PropertyNames.COMPONENT_ENCODINGS, component); CloudObject encodedCoder = CloudObject.forClassName("kind:windowed_value"); @@ -283,7 +322,7 @@ private ParallelInstruction makeWindowingSourceInstruction(Coder coder) { PropertyNames.COMPONENT_ENCODINGS, ImmutableList.of( timerCloudObject, - CloudObjects.asCloudObject(IntervalWindowCoder.of(), /*sdkComponents=*/ null))); + CloudObjects.asCloudObject(IntervalWindowCoder.of(), /* sdkComponents= */ null))); return new ParallelInstruction() .setSystemName(DEFAULT_SOURCE_SYSTEM_NAME) @@ -295,7 +334,7 @@ private ParallelInstruction makeWindowingSourceInstruction(Coder coder) { .setSpec(CloudObject.forClass(WindowingWindmillReader.class)) .setCodec(encodedCoder))) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setName(Long.toString(idGenerator.get())) .setCodec(encodedCoder) @@ -315,9 +354,9 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setCodec( CloudObjects.asCloudObject( WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), - /*sdkComponents=*/ null)))) + /* sdkComponents= */ null)))) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setName(Long.toString(idGenerator.get())) .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) @@ -325,7 +364,7 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setCodec( CloudObjects.asCloudObject( WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), - /*sdkComponents=*/ null)))); + /* sdkComponents= */ null)))); } private ParallelInstruction makeDoFnInstruction( @@ -360,9 +399,9 @@ private ParallelInstruction makeDoFnInstruction( .setNumOutputs(1) .setUserFn(spec) .setMultiOutputInfos( - Arrays.asList(new MultiOutputInfo().setTag(PropertyNames.OUTPUT)))) + Collections.singletonList(new MultiOutputInfo().setTag(PropertyNames.OUTPUT)))) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setName(PropertyNames.OUTPUT) .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) @@ -371,7 +410,7 @@ private ParallelInstruction makeDoFnInstruction( CloudObjects.asCloudObject( WindowedValue.getFullCoder( outputCoder, windowingStrategy.getWindowFn().windowCoder()), - /*sdkComponents=*/ null)))); + /* sdkComponents= */ null)))); } private ParallelInstruction makeDoFnInstruction( @@ -407,7 +446,7 @@ private ParallelInstruction makeSinkInstruction( .setCodec( CloudObjects.asCloudObject( WindowedValue.getFullCoder(coder, windowCoder), - /*sdkComponents=*/ null)))); + /* sdkComponents= */ null)))); } private ParallelInstruction makeSinkInstruction( @@ -493,26 +532,6 @@ private Windmill.GetWorkResponse buildSessionInput( .build(); } - private static final Function EMPTY_DATA_RESPONDER = - (GetDataRequest request) -> { - GetDataResponse.Builder builder = GetDataResponse.newBuilder(); - for (ComputationGetDataRequest compRequest : request.getRequestsList()) { - ComputationGetDataResponse.Builder compBuilder = - builder.addDataBuilder().setComputationId(compRequest.getComputationId()); - for (KeyedGetDataRequest keyRequest : compRequest.getRequestsList()) { - KeyedGetDataResponse.Builder keyBuilder = - compBuilder - .addDataBuilder() - .setKey(keyRequest.getKey()) - .setShardingKey(keyRequest.getShardingKey()); - keyBuilder.addAllValues(keyRequest.getValuesToFetchList()); - keyBuilder.addAllBags(keyRequest.getBagsToFetchList()); - keyBuilder.addAllWatermarkHolds(keyRequest.getWatermarkHoldsToFetchList()); - } - } - return builder.build(); - }; - private Windmill.GetWorkResponse makeInput(int index, long timestamp) throws Exception { return makeInput(index, timestamp, keyStringForIndex(index), DEFAULT_SHARDING_KEY); } @@ -552,7 +571,8 @@ private Windmill.GetWorkResponse makeInput( + " }" + "}", CoderUtils.encodeToByteArray( - CollectionCoder.of(IntervalWindow.getCoder()), Arrays.asList(DEFAULT_WINDOW))); + CollectionCoder.of(IntervalWindow.getCoder()), + Collections.singletonList(DEFAULT_WINDOW))); } /** @@ -684,8 +704,7 @@ private StreamingComputationConfig makeDefaultStreamingComputationConfig( return config; } - private ByteString addPaneTag(PaneInfo pane, byte[] windowBytes) - throws CoderException, IOException { + private ByteString addPaneTag(PaneInfo pane, byte[] windowBytes) throws IOException { ByteStringOutputStream output = new ByteStringOutputStream(); PaneInfo.PaneInfoCoder.INSTANCE.encode(pane, output, Context.OUTER); output.write(windowBytes); @@ -718,7 +737,7 @@ private StreamingDataflowWorker makeWorker( throws Exception { StreamingDataflowWorker worker = new StreamingDataflowWorker( - Arrays.asList(defaultMapTask(instructions)), + Collections.singletonList(defaultMapTask(instructions)), IntrinsicMapTaskExecutorFactory.defaultFactory(), mockWorkUnitClient, options, @@ -888,34 +907,6 @@ public void testHotKeyLoggingNotEnabled() throws Exception { verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(nullable(String.class), any()); } - static class BlockingFn extends DoFn implements TestRule { - - public static CountDownLatch blocker = new CountDownLatch(1); - public static Semaphore counter = new Semaphore(0); - public static AtomicInteger callCounter = new AtomicInteger(0); - - @ProcessElement - public void processElement(ProcessContext c) throws InterruptedException { - callCounter.incrementAndGet(); - counter.release(); - blocker.await(); - c.output(c.element()); - } - - @Override - public Statement apply(final Statement base, final Description description) { - return new Statement() { - @Override - public void evaluate() throws Throwable { - blocker = new CountDownLatch(1); - counter = new Semaphore(0); - callCounter = new AtomicInteger(); - base.evaluate(); - } - }; - } - } - @Test public void testIgnoreRetriedKeys() throws Exception { final int numIters = 4; @@ -1079,21 +1070,6 @@ public void testNumberOfWorkerHarnessThreadsIsHonored() throws Exception { BlockingFn.blocker.countDown(); } - static class KeyTokenInvalidFn extends DoFn, KV> { - - static boolean thrown = false; - - @ProcessElement - public void processElement(ProcessContext c) { - if (!thrown) { - thrown = true; - throw new KeyTokenInvalidException("key"); - } else { - c.output(c.element()); - } - } - } - @Test public void testKeyTokenInvalidException() throws Exception { if (streamingEngine) { @@ -1132,22 +1108,6 @@ public void testKeyTokenInvalidException() throws Exception { assertEquals(1, result.size()); } - static class LargeCommitFn extends DoFn, KV> { - - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element().getKey().equals("large_key")) { - StringBuilder s = new StringBuilder(); - for (int i = 0; i < 100; ++i) { - s.append("large_commit"); - } - c.output(KV.of(c.element().getKey(), s.toString())); - } else { - c.output(c.element()); - } - } - } - @Test public void testKeyCommitTooLargeException() throws Exception { KvCoder kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); @@ -1216,15 +1176,6 @@ public void testKeyCommitTooLargeException() throws Exception { assertTrue(foundErrors); } - static class ChangeKeysFn extends DoFn, KV> { - - @ProcessElement - public void processElement(ProcessContext c) { - KV elem = c.element(); - c.output(KV.of(elem.getKey() + "_" + elem.getValue(), elem.getValue())); - } - } - @Test public void testKeyChange() throws Exception { KvCoder kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); @@ -1280,23 +1231,6 @@ public void testKeyChange() throws Exception { } } - static class TestExceptionFn extends DoFn { - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - if (firstTime) { - firstTime = false; - try { - throw new Exception("Exception!"); - } catch (Exception e) { - throw new Exception("Another exception!", e); - } - } - } - - boolean firstTime = true; - } - @Test(timeout = 30000) public void testExceptions() throws Exception { if (streamingEngine) { @@ -1340,7 +1274,8 @@ public void testExceptions() throws Exception { + " }" + "}", CoderUtils.encodeToByteArray( - CollectionCoder.of(IntervalWindow.getCoder()), Arrays.asList(DEFAULT_WINDOW)))); + CollectionCoder.of(IntervalWindow.getCoder()), + Collections.singletonList(DEFAULT_WINDOW)))); StreamingDataflowWorker worker = makeWorker(instructions, createTestingPipelineOptions(server), true /* publishCounters */); @@ -1422,7 +1357,7 @@ public void testAssignWindows() throws Exception { .setNumOutputs(1) .setUserFn(spec)) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) @@ -1431,7 +1366,7 @@ public void testAssignWindows() throws Exception { CloudObjects.asCloudObject( WindowedValue.getFullCoder( StringUtf8Coder.of(), IntervalWindow.getCoder()), - /*sdkComponents=*/ null)))); + /* sdkComponents= */ null)))); List instructions = Arrays.asList( @@ -1539,7 +1474,7 @@ public void testMergeWindows() throws Exception { addObject( spec, WorkerPropertyNames.INPUT_CODER, - CloudObjects.asCloudObject(windowedKvCoder, /*sdkComponents=*/ null)); + CloudObjects.asCloudObject(windowedKvCoder, /* sdkComponents= */ null)); ParallelInstruction mergeWindowsInstruction = new ParallelInstruction() @@ -1552,14 +1487,14 @@ public void testMergeWindows() throws Exception { .setNumOutputs(1) .setUserFn(spec)) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setName("output") .setCodec( CloudObjects.asCloudObject( - windowedGroupedCoder, /*sdkComponents=*/ null)))); + windowedGroupedCoder, /* sdkComponents= */ null)))); List instructions = Arrays.asList( @@ -1749,7 +1684,7 @@ public void testMergeWindows() throws Exception { assertEquals( PaneInfo.createPane(true, true, Timing.ON_TIME), PaneInfoCoder.INSTANCE.decode(inStream)); assertEquals( - Arrays.asList(WINDOW_AT_ZERO), + Collections.singletonList(WINDOW_AT_ZERO), DEFAULT_WINDOW_COLLECTION_CODER.decode(inStream, Coder.Context.OUTER)); // Data was deleted @@ -1799,15 +1734,6 @@ public void testMergeWindows() throws Exception { assertEquals(0L, splitIntToLong(getCounter(counters, "WindmillShuffleBytesRead").getInteger())); } - static class PassthroughDoFn - extends DoFn>, KV>> { - - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element()); - } - } - @Test // Runs a merging windows test verifying stored state, holds and timers with caching due to // the first processing having is_new_key set. @@ -1835,7 +1761,7 @@ public void testMergeWindowsCaching() throws Exception { addObject( spec, WorkerPropertyNames.INPUT_CODER, - CloudObjects.asCloudObject(windowedKvCoder, /*sdkComponents=*/ null)); + CloudObjects.asCloudObject(windowedKvCoder, /* sdkComponents= */ null)); ParallelInstruction mergeWindowsInstruction = new ParallelInstruction() @@ -1848,14 +1774,14 @@ public void testMergeWindowsCaching() throws Exception { .setNumOutputs(1) .setUserFn(spec)) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setName("output") .setCodec( CloudObjects.asCloudObject( - windowedGroupedCoder, /*sdkComponents=*/ null)))); + windowedGroupedCoder, /* sdkComponents= */ null)))); List instructions = Arrays.asList( @@ -2048,7 +1974,7 @@ public void testMergeWindowsCaching() throws Exception { assertEquals( PaneInfo.createPane(true, true, Timing.ON_TIME), PaneInfoCoder.INSTANCE.decode(inStream)); assertEquals( - Arrays.asList(WINDOW_AT_ZERO), + Collections.singletonList(WINDOW_AT_ZERO), DEFAULT_WINDOW_COLLECTION_CODER.decode(inStream, Coder.Context.OUTER)); // Data was deleted @@ -2103,27 +2029,6 @@ public void testMergeWindowsCaching() throws Exception { assertEquals(4, stats.missCount()); } - static class Action { - - public Action(GetWorkResponse response) { - this.response = response; - } - - Action withHolds(WatermarkHold... holds) { - this.expectedHolds = holds; - return this; - } - - Action withTimers(Timer... timers) { - this.expectedTimers = timers; - return this; - } - - GetWorkResponse response; - Timer[] expectedTimers = new Timer[] {}; - WatermarkHold[] expectedHolds = new WatermarkHold[] {}; - } - // Helper for running tests for merging sessions based upon Actions consisting of GetWorkResponse // and expected timers and holds in the corresponding commit. All GetData requests are responded // to with empty state, relying on user worker caching to keep data written. @@ -2156,7 +2061,7 @@ private void runMergeSessionsActions(List actions) throws Exception { addObject( spec, WorkerPropertyNames.INPUT_CODER, - CloudObjects.asCloudObject(windowedKvCoder, /*sdkComponents=*/ null)); + CloudObjects.asCloudObject(windowedKvCoder, /* sdkComponents= */ null)); ParallelInstruction mergeWindowsInstruction = new ParallelInstruction() @@ -2169,14 +2074,14 @@ private void runMergeSessionsActions(List actions) throws Exception { .setNumOutputs(1) .setUserFn(spec)) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setName("output") .setCodec( CloudObjects.asCloudObject( - windowedGroupedCoder, /*sdkComponents=*/ null)))); + windowedGroupedCoder, /* sdkComponents= */ null)))); List instructions = Arrays.asList( @@ -2211,8 +2116,10 @@ private void runMergeSessionsActions(List actions) throws Exception { public void testMergeSessionWindows() throws Exception { // Test a single late window. runMergeSessionsActions( - Arrays.asList( - new Action(buildSessionInput(1, 40, 0, Arrays.asList(1L), Collections.EMPTY_LIST)) + Collections.singletonList( + new Action( + buildSessionInput( + 1, 40, 0, Collections.singletonList(1L), Collections.EMPTY_LIST)) .withHolds( buildHold("/gAAAAAAAAAsK/+uhold", -1, true), buildHold("/gAAAAAAAAAsK/+uextra", -1, true)) @@ -2226,7 +2133,9 @@ public void testMergeSessionWindows() throws Exception { // elements runMergeSessionsActions( Arrays.asList( - new Action(buildSessionInput(1, 0, 0, Arrays.asList(1L), Collections.EMPTY_LIST)) + new Action( + buildSessionInput( + 1, 0, 0, Collections.singletonList(1L), Collections.EMPTY_LIST)) .withHolds(buildHold("/gAAAAAAAAAsK/+uhold", 10, false)) .withTimers( buildWatermarkTimer("/s/gAAAAAAAAAsK/+0", 10), @@ -2237,12 +2146,14 @@ public void testMergeSessionWindows() throws Exception { 30, 0, Collections.EMPTY_LIST, - Arrays.asList(buildWatermarkTimer("/s/gAAAAAAAAAsK/+0", 10)))) + Collections.singletonList(buildWatermarkTimer("/s/gAAAAAAAAAsK/+0", 10)))) .withTimers(buildWatermarkTimer("/s/gAAAAAAAAAsK/+0", 3600010)) .withHolds( buildHold("/gAAAAAAAAAsK/+uhold", -1, true), buildHold("/gAAAAAAAAAsK/+uextra", -1, true)), - new Action(buildSessionInput(3, 30, 0, Arrays.asList(8L), Collections.EMPTY_LIST)) + new Action( + buildSessionInput( + 3, 30, 0, Collections.singletonList(8L), Collections.EMPTY_LIST)) .withTimers( buildWatermarkTimer("/s/gAAAAAAAABIR/+0", 3600017), buildWatermarkTimer("/s/gAAAAAAAAAsK/+0", 10, true), @@ -2250,7 +2161,9 @@ public void testMergeSessionWindows() throws Exception { .withHolds( buildHold("/gAAAAAAAAAsK/+uhold", -1, true), buildHold("/gAAAAAAAAAsK/+uextra", -1, true)), - new Action(buildSessionInput(4, 30, 0, Arrays.asList(31L), Collections.EMPTY_LIST)) + new Action( + buildSessionInput( + 4, 30, 0, Collections.singletonList(31L), Collections.EMPTY_LIST)) .withTimers( buildWatermarkTimer("/s/gAAAAAAAACkK/+0", 3600040), buildWatermarkTimer("/s/gAAAAAAAACkK/+0", 40)) @@ -2274,31 +2187,13 @@ public void testMergeSessionWindows() throws Exception { 50, 0, Collections.EMPTY_LIST, - Arrays.asList(buildWatermarkTimer("/s/gAAAAAAAACko/+0", 40)))) + Collections.singletonList(buildWatermarkTimer("/s/gAAAAAAAACko/+0", 40)))) .withTimers(buildWatermarkTimer("/s/gAAAAAAAACko/+0", 3600040)) .withHolds( buildHold("/gAAAAAAAAAsK/+uhold", -1, true), buildHold("/gAAAAAAAAAsK/+uextra", -1, true)))); } - private static CounterUpdate getCounter(Iterable counters, String name) { - for (CounterUpdate counter : counters) { - if (counter.getNameAndKind().getName().equals(name)) { - return counter; - } - } - return null; - } - - static class PrintFn extends DoFn>, String> { - - @ProcessElement - public void processElement(ProcessContext c) { - KV elem = c.element().getValue(); - c.output(elem.getKey() + ":" + elem.getValue()); - } - } - private List makeUnboundedSourcePipeline() throws Exception { return makeUnboundedSourcePipeline(1, new PrintFn()); } @@ -2316,7 +2211,7 @@ private List makeUnboundedSourcePipeline( ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), - /*sdkComponents=*/ null); + /* sdkComponents= */ null); return Arrays.asList( new ParallelInstruction() @@ -2329,7 +2224,7 @@ private List makeUnboundedSourcePipeline( new TestCountingSource(numMessagesPerShard), options) .setCodec(codec))) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setName("read_output") .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) @@ -2382,7 +2277,7 @@ public void testUnboundedSources() throws Exception { PaneInfo.NO_FIRING, CoderUtils.encodeToByteArray( CollectionCoder.of(GlobalWindow.Coder.INSTANCE), - Arrays.asList(GlobalWindow.INSTANCE)), + Collections.singletonList(GlobalWindow.INSTANCE)), parseCommitRequest( "key: \"0000000000000001\" " + "sharding_key: 1 " @@ -2457,7 +2352,7 @@ public void testUnboundedSources() throws Exception { assertThat(finalizeTracker, contains(0)); - assertEquals(null, getCounter(counters, "dataflow_input_size-computation")); + assertNull(getCounter(counters, "dataflow_input_size-computation")); // Test recovery (on a new key so fresh reader state). Counter is done. server @@ -2503,7 +2398,7 @@ public void testUnboundedSources() throws Exception { + "source_watermark: 1000") .build())); - assertEquals(null, getCounter(counters, "dataflow_input_size-computation")); + assertNull(getCounter(counters, "dataflow_input_size-computation")); } @Test @@ -2549,7 +2444,7 @@ public void testUnboundedSourcesDrain() throws Exception { PaneInfo.NO_FIRING, CoderUtils.encodeToByteArray( CollectionCoder.of(GlobalWindow.Coder.INSTANCE), - Arrays.asList(GlobalWindow.INSTANCE)), + Collections.singletonList(GlobalWindow.INSTANCE)), parseCommitRequest( "key: \"0000000000000001\" " + "sharding_key: 1 " @@ -2659,7 +2554,7 @@ public void testUnboundedSourceWorkRetry() throws Exception { PaneInfo.NO_FIRING, CoderUtils.encodeToByteArray( CollectionCoder.of(GlobalWindow.Coder.INSTANCE), - Arrays.asList(GlobalWindow.INSTANCE)), + Collections.singletonList(GlobalWindow.INSTANCE)), parseCommitRequest( "key: \"0000000000000001\" " + "sharding_key: 1 " @@ -2750,26 +2645,13 @@ public void testUnboundedSourceWorkRetry() throws Exception { assertThat(finalizeTracker, contains(0)); } - private static class MockWork extends StreamingDataflowWorker.Work { - - public MockWork(long workToken) { - super( - Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), - Instant::now, - Collections.emptyList()); - } - - @Override - public void run() {} - } - @Test public void testActiveWork() throws Exception { BoundedQueueExecutor mockExecutor = Mockito.mock(BoundedQueueExecutor.class); - StreamingDataflowWorker.ComputationState computationState = - new StreamingDataflowWorker.ComputationState( + ComputationState computationState = + new ComputationState( "computation", - defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + defaultMapTask(Collections.singletonList(makeSourceInstruction(StringUtf8Coder.of()))), mockExecutor, ImmutableMap.of(), null); @@ -2777,49 +2659,49 @@ public void testActiveWork() throws Exception { ShardedKey key1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1); ShardedKey key2 = ShardedKey.create(ByteString.copyFromUtf8("key2"), 2); - MockWork m1 = new MockWork(1); + Work m1 = createMockWork(1); assertTrue(computationState.activateWork(key1, m1)); Mockito.verify(mockExecutor).execute(m1, m1.getWorkItem().getSerializedSize()); - computationState.completeWork(key1, 1); + computationState.completeWorkAndScheduleNextWorkForKey(key1, 1); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify work queues. - MockWork m2 = new MockWork(2); + Work m2 = createMockWork(2); assertTrue(computationState.activateWork(key1, m2)); Mockito.verify(mockExecutor).execute(m2, m2.getWorkItem().getSerializedSize()); - MockWork m3 = new MockWork(3); + Work m3 = createMockWork(3); assertTrue(computationState.activateWork(key1, m3)); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify another key is a separate queue. - MockWork m4 = new MockWork(4); + Work m4 = createMockWork(4); assertTrue(computationState.activateWork(key2, m4)); Mockito.verify(mockExecutor).execute(m4, m4.getWorkItem().getSerializedSize()); - computationState.completeWork(key2, 4); + computationState.completeWorkAndScheduleNextWorkForKey(key2, 4); Mockito.verifyNoMoreInteractions(mockExecutor); - computationState.completeWork(key1, 2); + computationState.completeWorkAndScheduleNextWorkForKey(key1, 2); Mockito.verify(mockExecutor).forceExecute(m3, m3.getWorkItem().getSerializedSize()); - computationState.completeWork(key1, 3); + computationState.completeWorkAndScheduleNextWorkForKey(key1, 3); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify duplicate work dropped. - MockWork m5 = new MockWork(5); + Work m5 = createMockWork(5); computationState.activateWork(key1, m5); Mockito.verify(mockExecutor).execute(m5, m5.getWorkItem().getSerializedSize()); assertFalse(computationState.activateWork(key1, m5)); Mockito.verifyNoMoreInteractions(mockExecutor); - computationState.completeWork(key1, 5); + computationState.completeWorkAndScheduleNextWorkForKey(key1, 5); Mockito.verifyNoMoreInteractions(mockExecutor); } @Test public void testActiveWorkForShardedKeys() throws Exception { BoundedQueueExecutor mockExecutor = Mockito.mock(BoundedQueueExecutor.class); - StreamingDataflowWorker.ComputationState computationState = - new StreamingDataflowWorker.ComputationState( + ComputationState computationState = + new ComputationState( "computation", - defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + defaultMapTask(Collections.singletonList(makeSourceInstruction(StringUtf8Coder.of()))), mockExecutor, ImmutableMap.of(), null); @@ -2827,22 +2709,22 @@ public void testActiveWorkForShardedKeys() throws Exception { ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1); ShardedKey key1Shard2 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 2); - MockWork m1 = new MockWork(1); + Work m1 = createMockWork(1); assertTrue(computationState.activateWork(key1Shard1, m1)); Mockito.verify(mockExecutor).execute(m1, m1.getWorkItem().getSerializedSize()); - computationState.completeWork(key1Shard1, 1); + computationState.completeWorkAndScheduleNextWorkForKey(key1Shard1, 1); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify work queues. - MockWork m2 = new MockWork(2); + Work m2 = createMockWork(2); assertTrue(computationState.activateWork(key1Shard1, m2)); Mockito.verify(mockExecutor).execute(m2, m2.getWorkItem().getSerializedSize()); - MockWork m3 = new MockWork(3); + Work m3 = createMockWork(3); assertTrue(computationState.activateWork(key1Shard1, m3)); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify a different shard of key is a separate queue. - MockWork m4 = new MockWork(3); + Work m4 = createMockWork(3); assertFalse(computationState.activateWork(key1Shard1, m4)); Mockito.verifyNoMoreInteractions(mockExecutor); assertTrue(computationState.activateWork(key1Shard2, m4)); @@ -2850,7 +2732,7 @@ public void testActiveWorkForShardedKeys() throws Exception { // Verify duplicate work dropped assertFalse(computationState.activateWork(key1Shard2, m4)); - computationState.completeWork(key1Shard2, 3); + computationState.completeWorkAndScheduleNextWorkForKey(key1Shard2, 3); Mockito.verifyNoMoreInteractions(mockExecutor); } @@ -2873,10 +2755,10 @@ public void testMaxThreadMetric() throws Exception { .setDaemon(true) .build()); - StreamingDataflowWorker.ComputationState computationState = - new StreamingDataflowWorker.ComputationState( + ComputationState computationState = + new ComputationState( "computation", - defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + defaultMapTask(Collections.singletonList(makeSourceInstruction(StringUtf8Coder.of()))), executor, ImmutableMap.of(), null); @@ -2886,29 +2768,17 @@ public void testMaxThreadMetric() throws Exception { // overriding definition of MockWork to add sleep, which will help us keep track of how // long each work item takes to process and therefore let us manipulate how long the time // at which we're at max threads is. - MockWork m2 = - new MockWork(2) { - @Override - public void run() { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + Consumer sleepProcessWorkFn = + unused -> { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); } }; - MockWork m3 = - new MockWork(3) { - @Override - public void run() { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - }; + Work m2 = createMockWork(2, sleepProcessWorkFn); + Work m3 = createMockWork(3, sleepProcessWorkFn); assertTrue(computationState.activateWork(key1Shard1, m2)); assertTrue(computationState.activateWork(key1Shard1, m3)); @@ -2923,41 +2793,6 @@ public void run() { executor.shutdown(); } - static class TestExceptionInvalidatesCacheFn - extends DoFn>, String> { - - static boolean thrown = false; - - @StateId("int") - private final StateSpec> counter = StateSpecs.value(VarIntCoder.of()); - - @ProcessElement - public void processElement(ProcessContext c, @StateId("int") ValueState state) - throws Exception { - KV elem = c.element().getValue(); - if (elem.getValue() == 0) { - LOG.error("**** COUNTER 0 ****"); - assertEquals(null, state.read()); - state.write(42); - assertEquals((Integer) 42, state.read()); - } else if (elem.getValue() == 1) { - LOG.error("**** COUNTER 1 ****"); - assertEquals((Integer) 42, state.read()); - } else if (elem.getValue() == 2) { - if (!thrown) { - LOG.error("**** COUNTER 2 (will throw) ****"); - thrown = true; - throw new Exception("Exception!"); - } - LOG.error("**** COUNTER 2 (retry) ****"); - assertEquals((Integer) 42, state.read()); - } else { - throw new RuntimeException("only expecting values [0,2]"); - } - c.output(elem.getKey() + ":" + elem.getValue()); - } - } - @Test public void testExceptionInvalidatesCache() throws Exception { // We'll need to force the system to limit bundles to one message at a time. @@ -3003,7 +2838,7 @@ public void testExceptionInvalidatesCache() throws Exception { ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), - /*sdkComponents=*/ null); + /* sdkComponents= */ null); TestCountingSource counter = new TestCountingSource(3).withThrowOnFirstSnapshot(true); @@ -3018,7 +2853,7 @@ public void testExceptionInvalidatesCache() throws Exception { .setSource( CustomSources.serializeToCloudSource(counter, options).setCodec(codec))) .setOutputs( - Arrays.asList( + Collections.singletonList( new InstructionOutput() .setName("read_output") .setOriginalName(DEFAULT_OUTPUT_ORIGINAL_NAME) @@ -3168,21 +3003,6 @@ public void testExceptionInvalidatesCache() throws Exception { } } - private static class FanoutFn extends DoFn { - - @ProcessElement - public void processElement(ProcessContext c) { - StringBuilder builder = new StringBuilder(1000000); - for (int i = 0; i < 1000000; i++) { - builder.append(' '); - } - String largeString = builder.toString(); - for (int i = 0; i < 3000; i++) { - c.output(largeString); - } - } - } - @Test public void testHugeCommits() throws Exception { List instructions = @@ -3202,15 +3022,6 @@ public void testHugeCommits() throws Exception { worker.stop(); } - private static class SlowDoFn extends DoFn { - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - Thread.sleep(1000); - c.output(c.element()); - } - } - @Test public void testActiveWorkRefresh() throws Exception { List instructions = @@ -3235,290 +3046,54 @@ public void testActiveWorkRefresh() throws Exception { assertThat(server.numGetDataRequests(), greaterThan(0)); } - static class FakeClock implements Supplier { - private class FakeScheduledExecutor implements ScheduledExecutorService { - @Override - public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { - return true; - } - - @Override - public void execute(Runnable command) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public List> invokeAll(Collection> tasks) - throws InterruptedException { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public List> invokeAll( - Collection> tasks, long timeout, TimeUnit unit) - throws InterruptedException { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public T invokeAny(Collection> tasks) - throws ExecutionException, InterruptedException { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) - throws ExecutionException, InterruptedException, TimeoutException { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public boolean isShutdown() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public boolean isTerminated() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public void shutdown() {} - - @Override - public List shutdownNow() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public Future submit(Callable task) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public Future submit(Runnable task) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public Future submit(Runnable task, T result) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public ScheduledFuture scheduleAtFixedRate( - Runnable command, long initialDelay, long period, TimeUnit unit) { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public ScheduledFuture scheduleWithFixedDelay( - Runnable command, long initialDelay, long delay, TimeUnit unit) { - if (delay <= 0) { - throw new UnsupportedOperationException( - "Please supply a delay > 0 to scheduleWithFixedDelay"); - } - FakeClock.this.schedule( - Duration.millis(unit.toMillis(initialDelay)), - new Runnable() { - @Override - public void run() { - command.run(); - FakeClock.this.schedule(Duration.millis(unit.toMillis(delay)), this); - } - }); - FakeClock.this.sleep(Duration.ZERO); // Execute work that has an intial delay of zero. - return null; - } - } - - private static class Job implements Comparable { - final Instant when; - final Runnable work; - - Job(Instant when, Runnable work) { - this.when = when; - this.work = work; - } - - @Override - public int compareTo(Job job) { - return when.compareTo(job.when); - } - } - - private final PriorityQueue jobs = new PriorityQueue<>(); - private Instant now = Instant.now(); - - public ScheduledExecutorService newFakeScheduledExecutor(String unused) { - return new FakeScheduledExecutor(); - } - - @Override - public synchronized Instant get() { - return now; - } - - public synchronized void clear() { - jobs.clear(); - } - - public synchronized void sleep(Duration duration) { - if (duration.isShorterThan(Duration.ZERO)) { - throw new UnsupportedOperationException("Cannot sleep backwards in time"); - } - Instant endOfSleep = now.plus(duration); - while (true) { - Job job = jobs.peek(); - if (job == null || job.when.isAfter(endOfSleep)) { - break; - } - jobs.remove(); - now = job.when; - job.work.run(); - } - now = endOfSleep; - } - - private synchronized void schedule(Duration fromNow, Runnable work) { - jobs.add(new Job(now.plus(fromNow), work)); - } - } - - private static class FakeSlowDoFn extends DoFn { - private static FakeClock clock; // A static variable keeps this DoFn serializable. - private final Duration sleep; - - FakeSlowDoFn(FakeClock clock, Duration sleep) { - FakeSlowDoFn.clock = clock; - this.sleep = sleep; - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - clock.sleep(sleep); - c.output(c.element()); - } - } - @Test - public void testLatencyAttributionProtobufsPopulated() throws Exception { + public void testLatencyAttributionProtobufsPopulated() { FakeClock clock = new FakeClock(); - StreamingDataflowWorker.Work work = - new StreamingDataflowWorker.Work(null, clock, Collections.emptyList()) { - @Override - public void run() {} - }; + Work work = Work.create(null, clock, Collections.emptyList(), unused -> {}); clock.sleep(Duration.millis(10)); - work.setState(StreamingDataflowWorker.Work.State.PROCESSING); + work.setState(Work.State.PROCESSING); clock.sleep(Duration.millis(20)); - work.setState(StreamingDataflowWorker.Work.State.READING); + work.setState(Work.State.READING); clock.sleep(Duration.millis(30)); - work.setState(StreamingDataflowWorker.Work.State.PROCESSING); + work.setState(Work.State.PROCESSING); clock.sleep(Duration.millis(40)); - work.setState(StreamingDataflowWorker.Work.State.COMMIT_QUEUED); + work.setState(Work.State.COMMIT_QUEUED); clock.sleep(Duration.millis(50)); - work.setState(StreamingDataflowWorker.Work.State.COMMITTING); + work.setState(Work.State.COMMITTING); clock.sleep(Duration.millis(60)); Iterator it = work.getLatencyAttributions().iterator(); assertTrue(it.hasNext()); LatencyAttribution lat = it.next(); - assertTrue(lat.getState() == LatencyAttribution.State.QUEUED); - assertTrue(lat.getTotalDurationMillis() == 10); + assertSame(State.QUEUED, lat.getState()); + assertEquals(10, lat.getTotalDurationMillis()); assertTrue(it.hasNext()); lat = it.next(); - assertTrue(lat.getState() == LatencyAttribution.State.ACTIVE); - assertTrue(lat.getTotalDurationMillis() == 60); + assertSame(State.ACTIVE, lat.getState()); + assertEquals(60, lat.getTotalDurationMillis()); assertTrue(it.hasNext()); lat = it.next(); - assertTrue(lat.getState() == LatencyAttribution.State.READING); - assertTrue(lat.getTotalDurationMillis() == 30); + assertSame(State.READING, lat.getState()); + assertEquals(30, lat.getTotalDurationMillis()); assertTrue(it.hasNext()); lat = it.next(); - assertTrue(lat.getState() == LatencyAttribution.State.COMMITTING); - assertTrue(lat.getTotalDurationMillis() == 110); - assertTrue(!it.hasNext()); + assertSame(State.COMMITTING, lat.getState()); + assertEquals(110, lat.getTotalDurationMillis()); + assertFalse(it.hasNext()); } - // Aggregates LatencyAttribution data from active work refresh requests. - static class ActiveWorkRefreshSink { - private final Function responder; - private final Map> totalDurations = - new HashMap<>(); - - ActiveWorkRefreshSink(Function responder) { - this.responder = responder; - } + @Test + public void testLatencyAttributionToQueuedState() throws Exception { + final int workToken = 3232; // A unique id makes it easier to search logs. - Duration getLatencyAttributionDuration(long workToken, LatencyAttribution.State state) { - EnumMap durations = totalDurations.get(workToken); - return durations == null ? Duration.ZERO : durations.getOrDefault(state, Duration.ZERO); - } - - boolean isActiveWorkRefresh(GetDataRequest request) { - for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { - if (!computationRequest.getComputationId().equals(DEFAULT_COMPUTATION_ID)) { - return false; - } - for (KeyedGetDataRequest keyedRequest : computationRequest.getRequestsList()) { - if (keyedRequest.getWorkToken() == 0 - || keyedRequest.getShardingKey() != DEFAULT_SHARDING_KEY - || keyedRequest.getValuesToFetchCount() != 0 - || keyedRequest.getBagsToFetchCount() != 0 - || keyedRequest.getTagValuePrefixesToFetchCount() != 0 - || keyedRequest.getWatermarkHoldsToFetchCount() != 0) { - return false; - } - } - } - return true; - } - - GetDataResponse getData(GetDataRequest request) { - if (!isActiveWorkRefresh(request)) { - return responder.apply(request); - } - for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { - for (KeyedGetDataRequest keyedRequest : computationRequest.getRequestsList()) { - for (LatencyAttribution la : keyedRequest.getLatencyAttributionList()) { - EnumMap durations = - totalDurations.computeIfAbsent( - keyedRequest.getWorkToken(), - (Long workToken) -> - new EnumMap( - LatencyAttribution.State.class)); - Duration cur = Duration.millis(la.getTotalDurationMillis()); - durations.compute(la.getState(), (s, d) -> d == null || d.isShorterThan(cur) ? cur : d); - } - } - } - return EMPTY_DATA_RESPONDER.apply(request); - } - } - - @Test - public void testLatencyAttributionToQueuedState() throws Exception { - final int workToken = 3232; // A unique id makes it easier to search logs. - - FakeClock clock = new FakeClock(); - List instructions = - Arrays.asList( - makeSourceInstruction(StringUtf8Coder.of()), - makeDoFnInstruction( - new FakeSlowDoFn(clock, Duration.millis(1000)), 0, StringUtf8Coder.of()), - makeSinkInstruction(StringUtf8Coder.of(), 0)); + FakeClock clock = new FakeClock(); + List instructions = + Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeDoFnInstruction( + new FakeSlowDoFn(clock, Duration.millis(1000)), 0, StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); FakeWindmillServer server = new FakeWindmillServer(errorCollector); StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); @@ -3545,14 +3120,9 @@ public void testLatencyAttributionToQueuedState() throws Exception { worker.stop(); - assertTrue( - awrSink - .getLatencyAttributionDuration(workToken, LatencyAttribution.State.QUEUED) - .equals(Duration.millis(1000))); - assertTrue( - awrSink - .getLatencyAttributionDuration(workToken + 1, LatencyAttribution.State.QUEUED) - .equals(Duration.ZERO)); + assertEquals( + awrSink.getLatencyAttributionDuration(workToken, State.QUEUED), Duration.millis(1000)); + assertEquals(awrSink.getLatencyAttributionDuration(workToken + 1, State.QUEUED), Duration.ZERO); } @Test @@ -3587,22 +3157,8 @@ public void testLatencyAttributionToActiveState() throws Exception { worker.stop(); - assertTrue( - awrSink - .getLatencyAttributionDuration(workToken, LatencyAttribution.State.ACTIVE) - .equals(Duration.millis(1000))); - } - - // A DoFn that triggers a GetData request. - static class ReadingDoFn extends DoFn { - @StateId("int") - private final StateSpec> counter = StateSpecs.value(VarIntCoder.of()); - - @ProcessElement - public void processElement(ProcessContext c, @StateId("int") ValueState state) { - state.read(); - c.output(c.element()); - } + assertEquals( + awrSink.getLatencyAttributionDuration(workToken, State.ACTIVE), Duration.millis(1000)); } @Test @@ -3642,10 +3198,8 @@ public void testLatencyAttributionToReadingState() throws Exception { worker.stop(); - assertTrue( - awrSink - .getLatencyAttributionDuration(workToken, LatencyAttribution.State.READING) - .equals(Duration.millis(1000))); + assertEquals( + awrSink.getLatencyAttributionDuration(workToken, State.READING), Duration.millis(1000)); } @Test @@ -3685,10 +3239,8 @@ public void testLatencyAttributionToCommittingState() throws Exception { worker.stop(); - assertTrue( - awrSink - .getLatencyAttributionDuration(workToken, LatencyAttribution.State.COMMITTING) - .equals(Duration.millis(1000))); + assertEquals( + awrSink.getLatencyAttributionDuration(workToken, State.COMMITTING), Duration.millis(1000)); } @Test @@ -3742,24 +3294,6 @@ public void testLatencyAttributionPopulatedInCommitRequest() throws Exception { } } - /** For each input element, emits a large string. */ - private static class InflateDoFn extends DoFn>, String> { - - final int inflatedSize; - - /** For each input elements, outputs a string of this length */ - InflateDoFn(int inflatedSize) { - this.inflatedSize = inflatedSize; - } - - @ProcessElement - public void processElement(ProcessContext c) { - char[] chars = new char[inflatedSize]; - Arrays.fill(chars, ' '); - c.output(new String(chars)); - } - } - @Test public void testLimitOnOutputBundleSize() throws Exception { // This verifies that ReadOperation, StreamingModeExecutionContext, and windmill sinks @@ -3958,4 +3492,459 @@ public void testStuckCommit() throws Exception { .build(), removeDynamicFields(result.get(1L))); } + + static class BlockingFn extends DoFn implements TestRule { + + public static CountDownLatch blocker = new CountDownLatch(1); + public static Semaphore counter = new Semaphore(0); + public static AtomicInteger callCounter = new AtomicInteger(0); + + @ProcessElement + public void processElement(ProcessContext c) throws InterruptedException { + callCounter.incrementAndGet(); + counter.release(); + blocker.await(); + c.output(c.element()); + } + + @Override + public Statement apply(final Statement base, final Description description) { + return new Statement() { + @Override + public void evaluate() throws Throwable { + blocker = new CountDownLatch(1); + counter = new Semaphore(0); + callCounter = new AtomicInteger(); + base.evaluate(); + } + }; + } + } + + static class KeyTokenInvalidFn extends DoFn, KV> { + + static boolean thrown = false; + + @ProcessElement + public void processElement(ProcessContext c) { + if (!thrown) { + thrown = true; + throw new KeyTokenInvalidException("key"); + } else { + c.output(c.element()); + } + } + } + + static class LargeCommitFn extends DoFn, KV> { + + @ProcessElement + public void processElement(ProcessContext c) { + if (c.element().getKey().equals("large_key")) { + StringBuilder s = new StringBuilder(); + for (int i = 0; i < 100; ++i) { + s.append("large_commit"); + } + c.output(KV.of(c.element().getKey(), s.toString())); + } else { + c.output(c.element()); + } + } + } + + static class ChangeKeysFn extends DoFn, KV> { + + @ProcessElement + public void processElement(ProcessContext c) { + KV elem = c.element(); + c.output(KV.of(elem.getKey() + "_" + elem.getValue(), elem.getValue())); + } + } + + static class TestExceptionFn extends DoFn { + + boolean firstTime = true; + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + if (firstTime) { + firstTime = false; + try { + throw new Exception("Exception!"); + } catch (Exception e) { + throw new Exception("Another exception!", e); + } + } + } + } + + static class PassthroughDoFn + extends DoFn>, KV>> { + + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.element()); + } + } + + static class Action { + + GetWorkResponse response; + Timer[] expectedTimers = new Timer[] {}; + WatermarkHold[] expectedHolds = new WatermarkHold[] {}; + + public Action(GetWorkResponse response) { + this.response = response; + } + + Action withHolds(WatermarkHold... holds) { + this.expectedHolds = holds; + return this; + } + + Action withTimers(Timer... timers) { + this.expectedTimers = timers; + return this; + } + } + + static class PrintFn extends DoFn>, String> { + + @ProcessElement + public void processElement(ProcessContext c) { + KV elem = c.element().getValue(); + c.output(elem.getKey() + ":" + elem.getValue()); + } + } + + private static class MockWork { + Work create(long workToken) { + return Work.create( + Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), + Instant::now, + Collections.emptyList(), + work -> {}); + } + } + + static class TestExceptionInvalidatesCacheFn + extends DoFn>, String> { + + static boolean thrown = false; + + @StateId("int") + private final StateSpec> counter = StateSpecs.value(VarIntCoder.of()); + + @ProcessElement + public void processElement(ProcessContext c, @StateId("int") ValueState state) + throws Exception { + KV elem = c.element().getValue(); + if (elem.getValue() == 0) { + LOG.error("**** COUNTER 0 ****"); + assertNull(state.read()); + state.write(42); + assertEquals((Integer) 42, state.read()); + } else if (elem.getValue() == 1) { + LOG.error("**** COUNTER 1 ****"); + assertEquals((Integer) 42, state.read()); + } else if (elem.getValue() == 2) { + if (!thrown) { + LOG.error("**** COUNTER 2 (will throw) ****"); + thrown = true; + throw new Exception("Exception!"); + } + LOG.error("**** COUNTER 2 (retry) ****"); + assertEquals((Integer) 42, state.read()); + } else { + throw new RuntimeException("only expecting values [0,2]"); + } + c.output(elem.getKey() + ":" + elem.getValue()); + } + } + + private static class FanoutFn extends DoFn { + + @ProcessElement + public void processElement(ProcessContext c) { + StringBuilder builder = new StringBuilder(1000000); + for (int i = 0; i < 1000000; i++) { + builder.append(' '); + } + String largeString = builder.toString(); + for (int i = 0; i < 3000; i++) { + c.output(largeString); + } + } + } + + private static class SlowDoFn extends DoFn { + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + Thread.sleep(1000); + c.output(c.element()); + } + } + + static class FakeClock implements Supplier { + private final PriorityQueue jobs = new PriorityQueue<>(); + private Instant now = Instant.now(); + + public ScheduledExecutorService newFakeScheduledExecutor(String unused) { + return new FakeScheduledExecutor(); + } + + @Override + public synchronized Instant get() { + return now; + } + + public synchronized void clear() { + jobs.clear(); + } + + public synchronized void sleep(Duration duration) { + if (duration.isShorterThan(Duration.ZERO)) { + throw new UnsupportedOperationException("Cannot sleep backwards in time"); + } + Instant endOfSleep = now.plus(duration); + while (true) { + Job job = jobs.peek(); + if (job == null || job.when.isAfter(endOfSleep)) { + break; + } + jobs.remove(); + now = job.when; + job.work.run(); + } + now = endOfSleep; + } + + private synchronized void schedule(Duration fromNow, Runnable work) { + jobs.add(new Job(now.plus(fromNow), work)); + } + + private static class Job implements Comparable { + final Instant when; + final Runnable work; + + Job(Instant when, Runnable work) { + this.when = when; + this.work = work; + } + + @Override + public int compareTo(Job job) { + return when.compareTo(job.when); + } + } + + private class FakeScheduledExecutor implements ScheduledExecutorService { + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { + return true; + } + + @Override + public void execute(Runnable command) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public List> invokeAll(Collection> tasks) + throws InterruptedException { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public List> invokeAll( + Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public T invokeAny(Collection> tasks) + throws ExecutionException, InterruptedException { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) + throws ExecutionException, InterruptedException, TimeoutException { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public boolean isShutdown() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public boolean isTerminated() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public void shutdown() {} + + @Override + public List shutdownNow() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public Future submit(Callable task) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public Future submit(Runnable task) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public Future submit(Runnable task, T result) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public ScheduledFuture scheduleAtFixedRate( + Runnable command, long initialDelay, long period, TimeUnit unit) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public ScheduledFuture scheduleWithFixedDelay( + Runnable command, long initialDelay, long delay, TimeUnit unit) { + if (delay <= 0) { + throw new UnsupportedOperationException( + "Please supply a delay > 0 to scheduleWithFixedDelay"); + } + FakeClock.this.schedule( + Duration.millis(unit.toMillis(initialDelay)), + new Runnable() { + @Override + public void run() { + command.run(); + FakeClock.this.schedule(Duration.millis(unit.toMillis(delay)), this); + } + }); + FakeClock.this.sleep(Duration.ZERO); // Execute work that has an intial delay of zero. + return null; + } + } + } + + private static class FakeSlowDoFn extends DoFn { + private static FakeClock clock; // A static variable keeps this DoFn serializable. + private final Duration sleep; + + FakeSlowDoFn(FakeClock clock, Duration sleep) { + FakeSlowDoFn.clock = clock; + this.sleep = sleep; + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + clock.sleep(sleep); + c.output(c.element()); + } + } + + // Aggregates LatencyAttribution data from active work refresh requests. + static class ActiveWorkRefreshSink { + private final Function responder; + private final Map> totalDurations = + new HashMap<>(); + + ActiveWorkRefreshSink(Function responder) { + this.responder = responder; + } + + Duration getLatencyAttributionDuration(long workToken, LatencyAttribution.State state) { + EnumMap durations = totalDurations.get(workToken); + return durations == null ? Duration.ZERO : durations.getOrDefault(state, Duration.ZERO); + } + + boolean isActiveWorkRefresh(GetDataRequest request) { + for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { + if (!computationRequest.getComputationId().equals(DEFAULT_COMPUTATION_ID)) { + return false; + } + for (KeyedGetDataRequest keyedRequest : computationRequest.getRequestsList()) { + if (keyedRequest.getWorkToken() == 0 + || keyedRequest.getShardingKey() != DEFAULT_SHARDING_KEY + || keyedRequest.getValuesToFetchCount() != 0 + || keyedRequest.getBagsToFetchCount() != 0 + || keyedRequest.getTagValuePrefixesToFetchCount() != 0 + || keyedRequest.getWatermarkHoldsToFetchCount() != 0) { + return false; + } + } + } + return true; + } + + GetDataResponse getData(GetDataRequest request) { + if (!isActiveWorkRefresh(request)) { + return responder.apply(request); + } + for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { + for (KeyedGetDataRequest keyedRequest : computationRequest.getRequestsList()) { + for (LatencyAttribution la : keyedRequest.getLatencyAttributionList()) { + EnumMap durations = + totalDurations.computeIfAbsent( + keyedRequest.getWorkToken(), + (Long workToken) -> + new EnumMap( + LatencyAttribution.State.class)); + Duration cur = Duration.millis(la.getTotalDurationMillis()); + durations.compute(la.getState(), (s, d) -> d == null || d.isShorterThan(cur) ? cur : d); + } + } + } + return EMPTY_DATA_RESPONDER.apply(request); + } + } + + // A DoFn that triggers a GetData request. + static class ReadingDoFn extends DoFn { + @StateId("int") + private final StateSpec> counter = StateSpecs.value(VarIntCoder.of()); + + @ProcessElement + public void processElement(ProcessContext c, @StateId("int") ValueState state) { + state.read(); + c.output(c.element()); + } + } + + /** For each input element, emits a large string. */ + private static class InflateDoFn extends DoFn>, String> { + + final int inflatedSize; + + /** For each input elements, outputs a string of this length */ + InflateDoFn(int inflatedSize) { + this.inflatedSize = inflatedSize; + } + + @ProcessElement + public void processElement(ProcessContext c) { + char[] chars = new char[inflatedSize]; + Arrays.fill(chars, ' '); + c.output(new String(chars)); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java new file mode 100644 index 000000000000..1f3dee4b76ba --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.ActivateWorkResult; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ActiveWorkStateTest { + + private final WindmillStateCache.ForComputation computationStateCache = + mock(WindmillStateCache.ForComputation.class); + private Map> readOnlyActiveWork; + + private ActiveWorkState activeWorkState; + + private static ShardedKey shardedKey(String str, long shardKey) { + return ShardedKey.create(ByteString.copyFromUtf8(str), shardKey); + } + + private static Work emptyWork() { + return createWork(null); + } + + private static Work createWork(@Nullable Windmill.WorkItem workItem) { + return Work.create(workItem, Instant::now, Collections.emptyList(), unused -> {}); + } + + private static Work expiredWork(Windmill.WorkItem workItem) { + return Work.create(workItem, () -> Instant.EPOCH, Collections.emptyList(), unused -> {}); + } + + private static Windmill.WorkItem createWorkItem(long workToken) { + return Windmill.WorkItem.newBuilder() + .setKey(ByteString.copyFromUtf8("")) + .setShardingKey(1) + .setWorkToken(workToken) + .build(); + } + + @Before + public void setup() { + Map> readWriteActiveWorkMap = new HashMap<>(); + // Only use readOnlyActiveWork to verify internal behavior in reaction to exposed API calls. + readOnlyActiveWork = Collections.unmodifiableMap(readWriteActiveWorkMap); + activeWorkState = ActiveWorkState.forTesting(readWriteActiveWorkMap, computationStateCache); + } + + @Test + public void testActivateWorkForKey_EXECUTE_unknownKey() { + ActivateWorkResult activateWorkResult = + activeWorkState.activateWorkForKey(shardedKey("someKey", 1L), emptyWork()); + + assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult); + } + + @Test + public void testActivateWorkForKey_EXECUTE_emptyWorkQueueForKey() { + ShardedKey shardedKey = shardedKey("someKey", 1L); + long workToken = 1L; + + ActivateWorkResult activateWorkResult = + activeWorkState.activateWorkForKey(shardedKey, createWork(createWorkItem(workToken))); + + Optional nextWorkForKey = + activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, workToken); + + assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult); + assertEquals(Optional.empty(), nextWorkForKey); + assertThat(readOnlyActiveWork).doesNotContainKey(shardedKey); + } + + @Test + public void testActivateWorkForKey_DUPLICATE() { + long workToken = 10L; + ShardedKey shardedKey = shardedKey("someKey", 1L); + + // ActivateWork with the same shardedKey, and the same workTokens. + activeWorkState.activateWorkForKey(shardedKey, createWork(createWorkItem(workToken))); + ActivateWorkResult activateWorkResult = + activeWorkState.activateWorkForKey(shardedKey, createWork(createWorkItem(workToken))); + + assertEquals(ActivateWorkResult.DUPLICATE, activateWorkResult); + } + + @Test + public void testActivateWorkForKey_QUEUED() { + ShardedKey shardedKey = shardedKey("someKey", 1L); + + // ActivateWork with the same shardedKey, but different workTokens. + activeWorkState.activateWorkForKey(shardedKey, createWork(createWorkItem(1L))); + ActivateWorkResult activateWorkResult = + activeWorkState.activateWorkForKey(shardedKey, createWork(createWorkItem(2L))); + + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); + } + + @Test + public void testCompleteWorkAndGetNextWorkForKey_noWorkQueueForKey() { + assertEquals( + Optional.empty(), + activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey("someKey", 1L), 10L)); + } + + @Test + public void testCompleteWorkAndGetNextWorkForKey_currentWorkInQueueDoesNotMatchWorkToComplete() { + long workTokenToComplete = 1L; + + Work workInQueue = createWork(createWorkItem(2L)); + ShardedKey shardedKey = shardedKey("someKey", 1L); + + activeWorkState.activateWorkForKey(shardedKey, workInQueue); + activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, workTokenToComplete); + + assertEquals(1, readOnlyActiveWork.get(shardedKey).size()); + assertEquals(workInQueue, readOnlyActiveWork.get(shardedKey).peek()); + } + + @Test + public void testCompleteWorkAndGetNextWorkForKey_removesWorkFromQueueWhenComplete() { + long workTokenToComplete = 1L; + + Work activeWork = createWork(createWorkItem(workTokenToComplete)); + Work nextWork = createWork(createWorkItem(2L)); + ShardedKey shardedKey = shardedKey("someKey", 1L); + + activeWorkState.activateWorkForKey(shardedKey, activeWork); + activeWorkState.activateWorkForKey(shardedKey, nextWork); + activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, workTokenToComplete); + + assertEquals(nextWork, readOnlyActiveWork.get(shardedKey).peek()); + assertEquals(1, readOnlyActiveWork.get(shardedKey).size()); + assertFalse(readOnlyActiveWork.get(shardedKey).contains(activeWork)); + } + + @Test + public void testCompleteWorkAndGetNextWorkForKey_removesQueueIfNoWorkPresent() { + Work workInQueue = createWork(createWorkItem(1L)); + ShardedKey shardedKey = shardedKey("someKey", 1L); + + activeWorkState.activateWorkForKey(shardedKey, workInQueue); + activeWorkState.completeWorkAndGetNextWorkForKey( + shardedKey, workInQueue.getWorkItem().getWorkToken()); + + assertFalse(readOnlyActiveWork.containsKey(shardedKey)); + } + + @Test + public void testCompleteWorkAndGetNextWorkForKey_returnsWorkIfPresent() { + Work workToBeCompleted = createWork(createWorkItem(1L)); + Work nextWork = createWork(createWorkItem(2L)); + ShardedKey shardedKey = shardedKey("someKey", 1L); + + activeWorkState.activateWorkForKey(shardedKey, workToBeCompleted); + activeWorkState.activateWorkForKey(shardedKey, nextWork); + activeWorkState.completeWorkAndGetNextWorkForKey( + shardedKey, workToBeCompleted.getWorkItem().getWorkToken()); + + Optional nextWorkOpt = + activeWorkState.completeWorkAndGetNextWorkForKey( + shardedKey, workToBeCompleted.getWorkItem().getWorkToken()); + + assertTrue(nextWorkOpt.isPresent()); + assertSame(nextWork, nextWorkOpt.get()); + + Optional endOfWorkQueue = + activeWorkState.completeWorkAndGetNextWorkForKey( + shardedKey, nextWork.getWorkItem().getWorkToken()); + + assertFalse(endOfWorkQueue.isPresent()); + assertFalse(readOnlyActiveWork.containsKey(shardedKey)); + } + + @Test + public void testInvalidateStuckCommits() { + Map invalidatedCommits = new HashMap<>(); + + Work stuckWork1 = expiredWork(createWorkItem(1L)); + stuckWork1.setState(Work.State.COMMITTING); + Work stuckWork2 = expiredWork(createWorkItem(2L)); + stuckWork2.setState(Work.State.COMMITTING); + ShardedKey shardedKey1 = shardedKey("someKey", 1L); + ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); + + activeWorkState.activateWorkForKey(shardedKey1, stuckWork1); + activeWorkState.activateWorkForKey(shardedKey2, stuckWork2); + + activeWorkState.invalidateStuckCommits(Instant.now(), invalidatedCommits::put); + + assertThat(invalidatedCommits) + .containsEntry(shardedKey1, stuckWork1.getWorkItem().getWorkToken()); + assertThat(invalidatedCommits) + .containsEntry(shardedKey2, stuckWork2.getWorkItem().getWorkToken()); + verify(computationStateCache).invalidate(shardedKey1.key(), shardedKey1.shardingKey()); + verify(computationStateCache).invalidate(shardedKey2.key(), shardedKey2.shardingKey()); + } + + @Test + public void testGetKeysToRefresh() { + Instant refreshDeadline = Instant.now(); + + Work freshWork = createWork(createWorkItem(3L)); + Work refreshableWork1 = expiredWork(createWorkItem(1L)); + refreshableWork1.setState(Work.State.COMMITTING); + Work refreshableWork2 = expiredWork(createWorkItem(2L)); + refreshableWork2.setState(Work.State.COMMITTING); + ShardedKey shardedKey1 = shardedKey("someKey", 1L); + ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); + + activeWorkState.activateWorkForKey(shardedKey1, refreshableWork1); + activeWorkState.activateWorkForKey(shardedKey1, freshWork); + activeWorkState.activateWorkForKey(shardedKey2, refreshableWork2); + + ImmutableList requests = activeWorkState.getKeysToRefresh(refreshDeadline); + + ImmutableList expected = + ImmutableList.of( + GetDataRequestKeyShardingKeyAndWorkToken.from(shardedKey1, refreshableWork1), + GetDataRequestKeyShardingKeyAndWorkToken.from(shardedKey2, refreshableWork2)); + + ImmutableList actual = + requests.stream() + .map(GetDataRequestKeyShardingKeyAndWorkToken::from) + .collect(toImmutableList()); + + assertThat(actual).containsExactlyElementsIn(expected); + } + + @AutoValue + abstract static class GetDataRequestKeyShardingKeyAndWorkToken { + + private static GetDataRequestKeyShardingKeyAndWorkToken create( + ByteString key, long shardingKey, long workToken) { + return new AutoValue_ActiveWorkStateTest_GetDataRequestKeyShardingKeyAndWorkToken( + key, shardingKey, workToken); + } + + private static GetDataRequestKeyShardingKeyAndWorkToken from( + KeyedGetDataRequest keyedGetDataRequest) { + return create( + keyedGetDataRequest.getKey(), + keyedGetDataRequest.getShardingKey(), + keyedGetDataRequest.getWorkToken()); + } + + private static GetDataRequestKeyShardingKeyAndWorkToken from(ShardedKey shardedKey, Work work) { + return create(shardedKey.key(), shardedKey.shardingKey(), work.getWorkItem().getWorkToken()); + } + + abstract ByteString key(); + + abstract long shardingKey(); + + abstract long workToken(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/WeightBoundedQueueTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/WeightBoundedQueueTest.java new file mode 100644 index 000000000000..b2d98fb0e954 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/WeightBoundedQueueTest.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class WeightBoundedQueueTest { + private static final int MAX_WEIGHT = 10; + + @Test + public void testPut_hasCapacity() { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + + int insertedValue = 1; + + queue.put(insertedValue); + + assertEquals(insertedValue, queue.queuedElementsWeight()); + assertEquals(1, queue.size()); + assertEquals(insertedValue, (int) queue.poll()); + } + + @Test + public void testPut_noCapacity() throws InterruptedException { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + + // Insert value that takes all the capacity into the queue. + queue.put(MAX_WEIGHT); + + // Try to insert another value into the queue. This will block since there is no capacity in the + // queue. + Thread putThread = + new Thread( + () -> { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + queue.put(MAX_WEIGHT); + }); + putThread.start(); + + // Should only see the first value in the queue, since the queue is at capacity. thread2 + // should be blocked. + assertEquals(MAX_WEIGHT, queue.queuedElementsWeight()); + assertEquals(1, queue.size()); + + // Poll the queue, pulling off the only value inside and freeing up the capacity in the queue. + queue.poll(); + + // Wait for the putThread which was previously blocked due to the queue being at capacity. + putThread.join(); + + assertEquals(MAX_WEIGHT, queue.queuedElementsWeight()); + assertEquals(1, queue.size()); + } + + @Test + public void testPoll() { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + + int insertedValue1 = 1; + int insertedValue2 = 2; + + queue.put(insertedValue1); + queue.put(insertedValue2); + + assertEquals(insertedValue1 + insertedValue2, queue.queuedElementsWeight()); + assertEquals(2, queue.size()); + assertEquals(insertedValue1, (int) queue.poll()); + assertEquals(1, queue.size()); + } + + @Test + public void testPoll_withTimeout() throws InterruptedException { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + int pollWaitTimeMillis = 10000; + int insertedValue1 = 1; + + AtomicInteger pollResult = new AtomicInteger(); + Thread pollThread = + new Thread( + () -> { + int polled; + try { + polled = queue.poll(pollWaitTimeMillis, TimeUnit.MILLISECONDS); + pollResult.set(polled); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + pollThread.start(); + Thread.sleep(pollWaitTimeMillis / 100); + queue.put(insertedValue1); + pollThread.join(); + + assertEquals(insertedValue1, pollResult.get()); + } + + @Test + public void testPoll_withTimeout_timesOut() throws InterruptedException { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + int defaultPollResult = -10; + int pollWaitTimeMillis = 100; + int insertedValue1 = 1; + + // AtomicInteger default isn't null, so set it to a negative value and verify that it doesn't + // change. + AtomicInteger pollResult = new AtomicInteger(defaultPollResult); + + Thread pollThread = + new Thread( + () -> { + int polled; + try { + polled = queue.poll(pollWaitTimeMillis, TimeUnit.MILLISECONDS); + pollResult.set(polled); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + pollThread.start(); + Thread.sleep(pollWaitTimeMillis * 100); + queue.put(insertedValue1); + pollThread.join(); + + assertEquals(defaultPollResult, pollResult.get()); + } + + @Test + public void testPoll_emptyQueue() { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + + assertNull(queue.poll()); + } + + @Test + public void testTake() throws InterruptedException { + WeightedBoundedQueue queue = + WeightedBoundedQueue.create(MAX_WEIGHT, i -> Math.min(MAX_WEIGHT, i)); + + AtomicInteger value = new AtomicInteger(); + // Should block until value is available + Thread takeThread = + new Thread( + () -> { + try { + value.set(queue.take()); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + takeThread.start(); + + Thread.sleep(100); + queue.put(MAX_WEIGHT); + + takeThread.join(); + + assertEquals(MAX_WEIGHT, value.get()); + } +} From 5515f18b915f8b64c6a0d4bc2949787b7db939f3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 28 Sep 2023 13:51:02 -0400 Subject: [PATCH 28/64] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#28673) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.86 to 1.11.87. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.86...feature/s3/manager/v1.11.87) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e1ad4a9707e4..869db285534d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -33,7 +33,7 @@ require ( github.com/aws/aws-sdk-go-v2 v1.21.0 github.com/aws/aws-sdk-go-v2/config v1.18.42 github.com/aws/aws-sdk-go-v2/credentials v1.13.40 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.86 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 github.com/aws/smithy-go v1.14.2 github.com/docker/go-connections v0.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index d72d5ab27ba3..52512ffd424c 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -95,8 +95,8 @@ github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDu github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.86 h1:tnn/U5bz5flqoTCFSgRMEdg93ULR9Q6+tL5LkwjJ0DM= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.86/go.mod h1:TJGNZIhz3fsaQ6PU9roZacAEMMnG89X2UzaDblNoeNw= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 h1:e20ZrsgDPUXqg8+rZVuPwNSp6yniUN2Yr2tzFZ+Yvl0= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87/go.mod h1:0i0TAT6W+5i48QTlDU2KmY6U2hBZeY/LCP0wktya2oc= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 h1:22dGT7PneFMx4+b3pz7lMTRyN8ZKH7M2cW4GP9yUS2g= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41/go.mod h1:CrObHAuPneJBlfEJ5T3szXOUkLEThaGfvnhTf33buas= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 h1:SijA0mgjV8E+8G45ltVHs0fvKpTj8xmZJ3VwhGKtUSI= @@ -118,7 +118,6 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNN github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 h1:v0jkRigbSD6uOdwcaUQmgEwG1BkPfAPDqaeNt/29ghg= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4/go.mod h1:LhTyt8J04LL+9cIt7pYJ5lbS/U98ZmXovLOR/4LUsk8= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.39.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= From a4ee8548424b335cb3ffdf6aeda899fd1e3ba8ad Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Thu, 28 Sep 2023 20:31:08 +0200 Subject: [PATCH 29/64] assign highmem runner to beam_PostCommit_Python and to beam_PreCommit_Java_GCP_IO_Direct (#28719) --- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 7dde6ed150df..0a5758ac9366 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -53,7 +53,7 @@ env: jobs: beam_PostCommit_Python: name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [self-hosted, ubuntu-20.04, highmem] timeout-minutes: 240 strategy: fail-fast: false diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 840b753d8190..b7720682ce23 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -90,7 +90,7 @@ jobs: github.event_name == 'schedule' || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_GCP_IO_Direct PreCommit' - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [self-hosted, ubuntu-20.04, highmem] steps: - uses: actions/checkout@v4 - name: Setup repository From 2d0a52f9346ec0d72db59118424685c444647cc2 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Thu, 28 Sep 2023 13:06:02 -0700 Subject: [PATCH 30/64] Remove repeated test scenarios. (#28669) --- sdks/python/apache_beam/examples/wordcount_it_test.py | 10 +--------- .../dataflow_exercise_metrics_pipeline_test.py | 10 ---------- 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/examples/wordcount_it_test.py b/sdks/python/apache_beam/examples/wordcount_it_test.py index 4bd6e430420b..63f661dba212 100644 --- a/sdks/python/apache_beam/examples/wordcount_it_test.py +++ b/sdks/python/apache_beam/examples/wordcount_it_test.py @@ -45,6 +45,7 @@ class WordCountIT(unittest.TestCase): DEFAULT_CHECKSUM = '33535a832b7db6d78389759577d4ff495980b9c0' @pytest.mark.it_postcommit + @pytest.mark.it_validatescontainer def test_wordcount_it(self): self._run_wordcount_it(wordcount.run) @@ -89,11 +90,6 @@ def test_wordcount_impersonation_it(self): with auth._Credentials._credentials_lock: auth._Credentials._credentials_init = False - @pytest.mark.it_postcommit - @pytest.mark.it_validatescontainer - def test_wordcount_fnapi_it(self): - self._run_wordcount_it(wordcount.run, experiment='beam_fn_api') - @pytest.mark.it_validatescontainer def test_wordcount_it_with_prebuilt_sdk_container_local_docker(self): self._run_wordcount_it( @@ -108,10 +104,6 @@ def test_wordcount_it_with_prebuilt_sdk_container_cloud_build(self): experiment='beam_fn_api', prebuild_sdk_container_engine='cloud_build') - @pytest.mark.it_validatescontainer - def test_wordcount_it_with_use_sibling_sdk_workers(self): - self._run_wordcount_it(wordcount.run, experiment='use_sibling_sdk_workers') - def _run_wordcount_it(self, run_wordcount, **opts): test_pipeline = TestPipeline(is_integration_test=True) extra_opts = {} diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_metrics_pipeline_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_metrics_pipeline_test.py index 7bc871d87090..909c15896a26 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_metrics_pipeline_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_metrics_pipeline_test.py @@ -51,16 +51,6 @@ def test_metrics_it(self): dataflow_exercise_metrics_pipeline.metric_matchers()) self.assertFalse(errors, str(errors)) - @pytest.mark.it_postcommit - @pytest.mark.it_validatescontainer - @unittest.skip('https://github.com/apache/beam/issues/22605') - def test_metrics_fnapi_it(self): - result = self.run_pipeline(experiment='beam_fn_api') - errors = metric_result_matchers.verify_all( - result.metrics().all_metrics(), - dataflow_exercise_metrics_pipeline.metric_matchers()) - self.assertFalse(errors, str(errors)) - if __name__ == '__main__': unittest.main() From dc9bec8f7c955812f38f464666089745f41202ce Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 29 Sep 2023 02:46:38 +0600 Subject: [PATCH 31/64] Github Workflow Replacement for Jenkins Jobs, beam_PerformanceTests_Compressed_TextIOIT* (#28606) --- ...m_PerformanceTests_Compressed_TextIOIT.yml | 91 +++++++++++++++ ...formanceTests_Compressed_TextIOIT_HDFS.yml | 109 ++++++++++++++++++ .../config_Compressed_TextIOIT.txt | 27 +++++ .../config_Compressed_TextIOIT_HDFS.txt | 27 +++++ 4 files changed, 254 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml create mode 100644 .github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml new file mode 100644 index 000000000000..0e82c0fdf7d1 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests Compressed TextIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 1/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_Compressed_TextIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java CompressedTextIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_Compressed_TextIOIT"] + job_phrase: ["Run Java CompressedTextIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) +# The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_Compressed_TextIOIT_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml new file mode 100644 index 000000000000..78c9d3e8ab7a --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -0,0 +1,109 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PerformanceTests Compressed TextIOIT HDFS + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 1/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_Compressed_TextIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java CompressedTextIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_Compressed_TextIOIT_HDFS"] + job_phrase: ["Run Java CompressedTextIO Performance Test HDFS"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT +# The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_Compressed_TextIOIT_HDFS_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt b/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt new file mode 100644 index 000000000000..137eb56354a3 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=compressed_textioit_results +--influxMeasurement=compressed_textioit_results +--numberOfRecords=450000000 +--expectedHash=8a3de973354abc6fba621c6797cc0f06 +--datasetSize=1097840000 +--compressionType=GZIP +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt new file mode 100644 index 000000000000..9ad5137cefec --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=compressed_textioit_hdfs_results +--influxMeasurement=compressed_textioit_hdfs_results +--numberOfRecords=450000000 +--expectedHash=8a3de973354abc6fba621c6797cc0f06 +--datasetSize=1097840000 +--compressionType=GZIP +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file From a6de303307281e9ca51b1a9cbc6a74a71aec39ee Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 29 Sep 2023 02:48:15 +0600 Subject: [PATCH 32/64] Github Workflow Replacement for Jenkins Jobs, beam_PerformanceTests_ParquetIOIT* (#28582) --- .../beam_PerformanceTests_ParquetIOIT.yml | 91 +++++++++++++++ ...beam_PerformanceTests_ParquetIOIT_HDFS.yml | 109 ++++++++++++++++++ .../config_ParquetIOIT.txt | 26 +++++ .../config_ParquetIOIT_HDFS.txt | 26 +++++ 4 files changed, 252 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_ParquetIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml create mode 100644 .github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml new file mode 100644 index 000000000000..ceb540b16b1f --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests ParquetIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 3/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_ParquetIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java ParquetIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_ParquetIOIT"] + job_phrase: ["Run Java ParquetIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) +# The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.parquet.ParquetIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_ParquetIOIT_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml new file mode 100644 index 000000000000..d0c40599eb62 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -0,0 +1,109 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PerformanceTests ParquetIOIT HDFS + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 3/12 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_ParquetIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java ParquetIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_ParquetIOIT_HDFS"] + job_phrase: ["Run Java ParquetIO Performance Test HDFS"] + 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: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT +# The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.parquet.ParquetIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_ParquetIOIT_HDFS_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt b/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt new file mode 100644 index 000000000000..10dc0eba73da --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt @@ -0,0 +1,26 @@ +# 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. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=parquetioit_results +--influxMeasurement=parquetioit_results +--numberOfRecords=225000000 +--expectedHash=2f9f5ca33ea464b25109c0297eb6aecb +--datasetSize=1087370000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt new file mode 100644 index 000000000000..ecb725cc12f7 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt @@ -0,0 +1,26 @@ +# 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. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=parquetioit_hdfs_results +--influxMeasurement=parquetioit_hdfs_results +--numberOfRecords=225000000 +--expectedHash=2f9f5ca33ea464b25109c0297eb6aecb +--datasetSize=1087370000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file From a9516ba18a7569524fd4f8092caeb12ff4c02bf6 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 28 Sep 2023 16:50:24 -0400 Subject: [PATCH 33/64] Add arm tests to postcommit suite (#28644) * Add arm tests to postcommit suite * Add separate arm suite * Temporarily add pull_request target * Remove pull request trigger * Temporarily just run on workflow_dispatch * Add to readme --- .github/workflows/README.md | 3 +- .../workflows/beam_PostCommit_Python_Arm.yml | 103 ++++++++++++++++++ sdks/python/scripts/run_integration_test.sh | 10 ++ .../python/test-suites/dataflow/common.gradle | 20 ++++ 4 files changed, 135 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/beam_PostCommit_Python_Arm.yml diff --git a/.github/workflows/README.md b/.github/workflows/README.md index f360bc15ae28..03479e5c4c3b 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -246,6 +246,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit PortableJar Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | N/A |`Run PortableJar_Flink PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | | [ PostCommit PortableJar Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | N/A |`Run PortableJar_Spark PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | | [ PostCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | +| [ PostCommit Python Arm](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit Arm (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Arm.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | | [ PostCommit Python Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | N/A |`Run Python Examples_Dataflow`| [![.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | | [ PostCommit Python Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Examples_Direct (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml) | | [ PostCommit Python Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | ['3.8','3.11'] |`Run Python Examples_Flink (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | @@ -346,4 +347,4 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | | [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | | [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | -| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | \ No newline at end of file +| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml new file mode 100644 index 000000000000..74eb8e8290d8 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -0,0 +1,103 @@ +# 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 Python Arm + +on: + # issue_comment: + # types: [created] + # schedule: + # - cron: '0 */6 * * *' + workflow_dispatch: + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Python_Arm: + name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + strategy: + fail-fast: false + matrix: + job_name: [beam_PostCommit_Python_Arm] + job_phrase: [Run Python PostCommit Arm] + python_version: ['3.8', '3.9', '3.10', '3.11'] + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + startsWith(github.event.comment.body, 'Run Python PostCommit Arm') + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) + - name: Install Python + uses: actions/setup-python@v4 + with: + python-version: ${{matrix.python_version}} + - name: Install docker compose + run: | + sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose + sudo chmod +x /usr/local/bin/docker-compose + - name: Set PY_VER_CLEAN + id: set_py_ver_clean + run: | + PY_VER=${{ matrix.python_version }} + PY_VER_CLEAN=${PY_VER//.} + echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT + - name: run PostCommit Python ${{ matrix.python_version }} script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:postCommitArmIT + arguments: | + -PuseWheelDistribution \ + -PpythonVersion=${{ matrix.python_version }} \ + env: + CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} + - name: Archive code coverage results + uses: actions/upload-artifact@v3 + with: + name: python-code-coverage-report + path: "**/pytest*.xml" diff --git a/sdks/python/scripts/run_integration_test.sh b/sdks/python/scripts/run_integration_test.sh index 4f29ed5a4ad9..6ad592080ae2 100755 --- a/sdks/python/scripts/run_integration_test.sh +++ b/sdks/python/scripts/run_integration_test.sh @@ -78,6 +78,7 @@ KMS_KEY_NAME="projects/apache-beam-testing/locations/global/keyRings/beam-it/cry SUITE="" COLLECT_MARKERS= REQUIREMENTS_FILE="" +ARCH="" # Default test (pytest) options. # Run WordCountIT.test_wordcount_it by default if no test options are @@ -163,6 +164,11 @@ case $key in shift # past argument shift # past value ;; + --arch) + ARCH="$2" + shift # past argument + shift # past value + ;; *) # unknown option echo "Unknown option: $1" exit 1 @@ -234,6 +240,10 @@ if [[ -z $PIPELINE_OPTS ]]; then opts+=("--streaming") fi + if [[ "$ARCH" == "ARM" ]]; then + opts+=("--machine_type=t2a-standard-1") + fi + if [[ ! -z "$KMS_KEY_NAME" ]]; then opts+=( "--kms_key_name=$KMS_KEY_NAME" diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index f907a728c0b5..7766cf3a377c 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -143,6 +143,26 @@ task postCommitIT { } } +task postCommitArmIT { + dependsOn 'initializeForDataflowJob' + + doLast { + def testOpts = basicPytestOpts + ["--numprocesses=8", "--dist=loadfile"] + def argMap = [ + "test_opts": testOpts, + "sdk_location": project.ext.sdkLocation, + "suite": "postCommitIT-df${pythonVersionSuffix}", + "collect": "it_postcommit", + "arch": "ARM" + ] + def cmdArgs = mapToArgString(argMap) + exec { + executable 'sh' + args '-c', ". ${envdir}/bin/activate && ${runScriptsDir}/run_integration_test.sh $cmdArgs" + } + } +} + task postCommitSickbay { dependsOn 'initializeForDataflowJob' From a46bc12a256dcaa3ae2cc9e5d6fdcaa82b59738b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 28 Sep 2023 14:25:59 -0700 Subject: [PATCH 34/64] Ensure configuration schema used to decode configuration. (#28727) Just because schemas are compatible doesn't mean that they're identical (e.g. up to field ordering). This is important as schema generation in Java is not always determanistic, so even if the payload is encoded with the exact schema that was previously provided it might not agree when the request comes in. --- ...pansionServiceSchemaTransformProvider.java | 3 +- ...ionServiceSchemaTransformProviderTest.java | 111 +++++++++++------- 2 files changed, 72 insertions(+), 42 deletions(-) diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java index 8d74f2f6117a..ead1fa67dc98 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java @@ -129,8 +129,7 @@ public PTransform getTransform(FunctionSpec spec) { Row configRow; try { configRow = - RowCoder.of(provider.configurationSchema()) - .decode(payload.getConfigurationRow().newInput()); + RowCoder.of(configSchemaFromRequest).decode(payload.getConfigurationRow().newInput()); } catch (IOException e) { throw new RuntimeException("Error decoding payload", e); } diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java index 141d2b48b105..d7a665eabe0f 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java @@ -19,9 +19,9 @@ import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import com.google.auto.service.AutoService; -import java.io.IOException; import java.util.ArrayList; import java.util.List; import org.apache.beam.model.expansion.v1.ExpansionApi; @@ -32,6 +32,7 @@ import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.schemas.JavaFieldSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; @@ -48,10 +49,11 @@ import org.apache.beam.sdk.transforms.InferableFunction; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.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.Iterables; @@ -74,6 +76,13 @@ public class ExpansionServiceSchemaTransformProviderTest { Field.of("str1", FieldType.STRING), Field.of("str2", FieldType.STRING)); + private static final Schema TEST_SCHEMATRANSFORM_EQUIVALENT_CONFIG_SCHEMA = + Schema.of( + Field.of("str2", FieldType.STRING), + Field.of("str1", FieldType.STRING), + Field.of("int2", FieldType.INT32), + Field.of("int1", FieldType.INT32)); + private ExpansionService expansionService = new ExpansionService(); @DefaultSchema(JavaFieldSchema.class) @@ -344,31 +353,13 @@ public void testSchemaTransformExpansion() { .withFieldValue("str2", "bbb") .build(); - ByteStringOutputStream outputStream = new ByteStringOutputStream(); - try { - SchemaCoder.of(configRow.getSchema()).encode(configRow, outputStream); - } catch (IOException e) { - throw new RuntimeException(e); - } - - ExternalTransforms.SchemaTransformPayload payload = - ExternalTransforms.SchemaTransformPayload.newBuilder() - .setIdentifier("dummy_id") - .setConfigurationRow(outputStream.toByteString()) - .setConfigurationSchema( - SchemaTranslation.schemaToProto(TEST_SCHEMATRANSFORM_CONFIG_SCHEMA, true)) - .build(); - ExpansionApi.ExpansionRequest request = ExpansionApi.ExpansionRequest.newBuilder() .setComponents(pipelineProto.getComponents()) .setTransform( RunnerApi.PTransform.newBuilder() .setUniqueName(TEST_NAME) - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(getUrn(ExpansionMethods.Enum.SCHEMA_TRANSFORM)) - .setPayload(payload.toByteString())) + .setSpec(createSpec("dummy_id", configRow)) .putInputs("input1", inputPcollId)) .setNamespace(TEST_NAMESPACE) .build(); @@ -403,35 +394,18 @@ public void testSchemaTransformExpansionMultiInputMultiOutput() { .withFieldValue("str2", "bbb") .build(); - ByteStringOutputStream outputStream = new ByteStringOutputStream(); - try { - SchemaCoder.of(configRow.getSchema()).encode(configRow, outputStream); - } catch (IOException e) { - throw new RuntimeException(e); - } - - ExternalTransforms.SchemaTransformPayload payload = - ExternalTransforms.SchemaTransformPayload.newBuilder() - .setIdentifier("dummy_id_multi_input_multi_output") - .setConfigurationRow(outputStream.toByteString()) - .setConfigurationSchema( - SchemaTranslation.schemaToProto(TEST_SCHEMATRANSFORM_CONFIG_SCHEMA, true)) - .build(); - ExpansionApi.ExpansionRequest request = ExpansionApi.ExpansionRequest.newBuilder() .setComponents(pipelineProto.getComponents()) .setTransform( RunnerApi.PTransform.newBuilder() .setUniqueName(TEST_NAME) - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(getUrn(ExpansionMethods.Enum.SCHEMA_TRANSFORM)) - .setPayload(payload.toByteString())) + .setSpec(createSpec("dummy_id_multi_input_multi_output", configRow)) .putInputs("input1", inputPcollIds.get(0)) .putInputs("input2", inputPcollIds.get(1))) .setNamespace(TEST_NAMESPACE) .build(); + ExpansionApi.ExpansionResponse response = expansionService.expand(request); RunnerApi.PTransform expandedTransform = response.getTransform(); @@ -440,4 +414,61 @@ public void testSchemaTransformExpansionMultiInputMultiOutput() { assertEquals(2, expandedTransform.getOutputsCount()); verifyLeafTransforms(response, 2); } + + @Test + public void testSchematransformEquivalentConfigSchema() throws CoderException { + Row configRow = + Row.withSchema(TEST_SCHEMATRANSFORM_CONFIG_SCHEMA) + .withFieldValue("int1", 111) + .withFieldValue("int2", 222) + .withFieldValue("str1", "aaa") + .withFieldValue("str2", "bbb") + .build(); + + RunnerApi.FunctionSpec spec = createSpec("dummy_id", configRow); + + Row equivalentConfigRow = + Row.withSchema(TEST_SCHEMATRANSFORM_EQUIVALENT_CONFIG_SCHEMA) + .withFieldValue("int1", 111) + .withFieldValue("int2", 222) + .withFieldValue("str1", "aaa") + .withFieldValue("str2", "bbb") + .build(); + + RunnerApi.FunctionSpec equivalentSpec = createSpec("dummy_id", equivalentConfigRow); + + assertNotEquals(spec.getPayload(), equivalentSpec.getPayload()); + + TestSchemaTransform transform = + (TestSchemaTransform) ExpansionServiceSchemaTransformProvider.of().getTransform(spec); + TestSchemaTransform equivalentTransform = + (TestSchemaTransform) + ExpansionServiceSchemaTransformProvider.of().getTransform(equivalentSpec); + + assertEquals(transform.int1, equivalentTransform.int1); + assertEquals(transform.int2, equivalentTransform.int2); + assertEquals(transform.str1, equivalentTransform.str1); + assertEquals(transform.str2, equivalentTransform.str2); + } + + private RunnerApi.FunctionSpec createSpec(String identifier, Row configRow) { + byte[] encodedRow; + try { + encodedRow = CoderUtils.encodeToByteArray(SchemaCoder.of(configRow.getSchema()), configRow); + } catch (CoderException e) { + throw new RuntimeException(e); + } + + ExternalTransforms.SchemaTransformPayload payload = + ExternalTransforms.SchemaTransformPayload.newBuilder() + .setIdentifier(identifier) + .setConfigurationRow(ByteString.copyFrom(encodedRow)) + .setConfigurationSchema(SchemaTranslation.schemaToProto(configRow.getSchema(), true)) + .build(); + + return RunnerApi.FunctionSpec.newBuilder() + .setUrn(getUrn(ExpansionMethods.Enum.SCHEMA_TRANSFORM)) + .setPayload(payload.toByteString()) + .build(); + } } From fbde88dca3e965e98b3cecd5d4953edd043dbe2d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lu=C3=ADs=20Bianchin?= Date: Fri, 29 Sep 2023 16:21:54 +0200 Subject: [PATCH 35/64] Update flogger-system-backend to 0.7.4 (#28585) Released a long time ago: https://github.com/google/flogger/releases/tag/flogger-0.7.4 Also aligned with version from hadoop-connectors/gcsio: https://github.com/GoogleCloudDataproc/hadoop-connectors/blame/757c0853720a6ff4216d706063ca3ab44a9588ba/pom.xml#L94 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 4f26534c533a..c31482d577e0 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -708,7 +708,7 @@ class BeamModulePlugin implements Plugin { dbcp2 : "org.apache.commons:commons-dbcp2:$dbcp2_version", error_prone_annotations : "com.google.errorprone:error_prone_annotations:$errorprone_version", failsafe : "dev.failsafe:failsafe:3.3.0", - flogger_system_backend : "com.google.flogger:flogger-system-backend:0.7.3", + flogger_system_backend : "com.google.flogger:flogger-system-backend:0.7.4", gax : "com.google.api:gax", // google_cloud_platform_libraries_bom sets version gax_grpc : "com.google.api:gax-grpc", // google_cloud_platform_libraries_bom sets version gax_grpc_test : "com.google.api:gax-grpc:$gax_version:testlib", // google_cloud_platform_libraries_bom sets version From 47ba976b82d78e648018c84fb7566b7bd96c933e Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Fri, 29 Sep 2023 18:51:16 +0400 Subject: [PATCH 36/64] Add GitHub Workflow Replacements for Jenkins job_PerformanceTests_TextIOIT (#28586) --- .../beam_PerformanceTests_TextIOIT.yml | 91 +++++++++++++++ .../beam_PerformanceTests_TextIOIT_HDFS.yml | 104 ++++++++++++++++++ .../beam_PerformanceTests_TextIOIT_Python.yml | 94 ++++++++++++++++ .../textIOIT.txt | 26 +++++ .../textIOIT_HDFS.txt | 26 +++++ .../textIOIT_Python.txt | 28 +++++ 6 files changed, 369 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_TextIOIT.yml create mode 100644 .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml create mode 100644 .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml create mode 100644 .github/workflows/performance-tests-job-configs/textIOIT.txt create mode 100644 .github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt create mode 100644 .github/workflows/performance-tests-job-configs/textIOIT_Python.txt diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml new file mode 100644 index 000000000000..9daa2b29dd2a --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -0,0 +1,91 @@ +# 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: PerformanceTests TextIOIT + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 7,19 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_TextIOIT: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java TextIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_TextIOIT"] + job_phrase: ["Run Java TextIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT.txt + arguments: | + --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}} + # The env variable is created and populated in the test-arguments-action as "beam_PerformanceTests_TextIOIT_test_arguments_1" + - name: Run Java TextIO Performance Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=gcs \ + -DintegrationTestRunner=dataflow \ + '-DintegrationTestPipelineOptions=[${{env.beam_PerformanceTests_TextIOIT_test_arguments_1}}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml new file mode 100644 index 000000000000..a98379b281a9 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -0,0 +1,104 @@ +# 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: PerformanceTests TextIOIT HDFS + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 7,19 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_TextIOIT_HDFS: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java TextIO Performance Test HDFS' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_TextIOIT_HDFS"] + job_phrase: ["Run Java TextIO Performance Test HDFS"] + 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: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Hadoop + id: install_hadoop + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml + kubectl wait svc/hadoop --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc hadoop -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo hadoop_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt + arguments: | + --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ + --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + # The env variable is created and populated in the test-arguments-action as "beam_PerformanceTests_TextIOIT_HDFS_test_arguments_1" + - name: Run Java TextIO Performance Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:file-based-io-tests:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.text.TextIOIT \ + --info \ + -Dfilesystem=hdfs \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{env.beam_PerformanceTests_TextIOIT_HDFS_test_arguments_1}}]' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml new file mode 100644 index 000000000000..cb2b7fb34a9f --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -0,0 +1,94 @@ +# 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: PerformanceTests TextIOIT Python + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 8,20 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_TextIOIT_Python: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Python TextIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_TextIOIT_Python"] + job_phrase: ["Run Python TextIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt + arguments: | + --filename_prefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}} + # The env variable is created and populated in the test-arguments-action as "beam_PerformanceTests_TextIOIT_Python_test_arguments_1" + - name: Run Python TextIO Performance Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.io.filebasedio_perf_test \ + -Prunner=DataflowRunner \ + '-PloadTest.args=${{env.beam_PerformanceTests_TextIOIT_Python_test_arguments_1}}' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/textIOIT.txt b/.github/workflows/performance-tests-job-configs/textIOIT.txt new file mode 100644 index 000000000000..dab9f5b082bd --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/textIOIT.txt @@ -0,0 +1,26 @@ +# 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. + +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests +--bigQueryDataset=beam_performance +--bigQueryTable=textioit_results +--influxMeasurement=textioit_results +--numberOfRecords=25000000 +--expectedHash=f8453256ccf861e8a312c125dfe0e436 +--datasetSize=1062290000 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt b/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt new file mode 100644 index 000000000000..0344ed1a57d2 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt @@ -0,0 +1,26 @@ +# 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. + +--bigQueryDataset=beam_performance +--bigQueryTable=textioit_hdfs_results +--influxMeasurement=textioit_hdfs_results +--numberOfRecords=25000000 +--expectedHash=f8453256ccf861e8a312c125dfe0e436 +--datasetSize=1062290000 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--runner=DataflowRunner +--tempRoot=gs://temp-storage-for-perf-tests \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt b/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt new file mode 100644 index 000000000000..58434aa61847 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--runner=DataflowRunner +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/ +--publish_to_big_query=true +--metrics_dataset=beam_performance +--metrics_table=python_textio_1GB_results +--influx_measurement=python_textio_1GB_results +--test_class=TextIOPerfTest +--input_options=''{\\"num_records\\":25000000,\\"key_size\\":9,\\"value_size\\":21,\\"algorithm\\":\\"lcg\\"}'' +--dataset_size=1050000000 +--num_workers=5 +--autoscaling_algorithm=NONE \ No newline at end of file From e7ec5fe8daf0070ce209408dc8be391e38067fde Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 29 Sep 2023 18:52:52 +0400 Subject: [PATCH 37/64] Add Load Tests GBK Flink Batch Go workflow (#28494) * Add Load Tests GBK Flink Batch Go workflow * Refactoring * Refactoring * Refactoring --- .../beam_LoadTests_Go_GBK_Flink_Batch.yml | 164 ++++++++++++++++++ .../go_GBK_Flink_Batch_100b.txt | 26 +++ .../go_GBK_Flink_Batch_100kb.txt | 26 +++ .../go_GBK_Flink_Batch_10b.txt | 26 +++ .../go_GBK_Flink_Batch_Fanout_4.txt | 26 +++ .../go_GBK_Flink_Batch_Fanout_8.txt | 26 +++ .../go_GBK_Flink_Batch_Reiteration_10KB.txt | 26 +++ 7 files changed, 320 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt create mode 100644 .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml new file mode 100644 index 000000000000..7c4d95738a09 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -0,0 +1,164 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Go GBK Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '20 1 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-go-gbk-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-go-gbk-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Go_GBK_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Go GBK Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Go_GBK_Flink_Batch"] + job_phrase: ["Run Load Tests Go GBK Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt + arguments: | + --job_name=load-tests-go-flink-batch-gbk-$(date '+%m%d%H%M%S' --utc) + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run GBK Flink Batch Go Load Test 1 (10 b records) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_1 }}' \ + - name: run GBK Flink Batch Go Load Test 2 (100 b records) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_2 }}' \ + - name: run GBK Flink Batch Go Load Test 3 (100 kb records) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_3 }}' \ + - name: run GBK Flink Batch Go Load Test 6 (reiterate 4 times 10 kb) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_6 }}' \ + - name: Restart Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh restart + - name: run GBK Flink Batch Go Load Test 4 (fanout 4) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_4 }}' \ + - name: run GBK Flink Batch Go Load Test 5 (fanout 8) + timeout-minutes: 120 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=group_by_key \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Flink_Batch_test_arguments_5 }}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt new file mode 100644 index 000000000000..78f99fb4e9cc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_2 +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90}'' +--iterations=1 +--fanout=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt new file mode 100644 index 000000000000..5772a27ca184 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_3 +--iterations=1 +--fanout=1 +--parallelism=5 +--input_options=''{\"num_records\":20000,\"key_size\":10000,\"value_size\":90000}'' +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt new file mode 100644 index 000000000000..e826fbe10dae --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_1 +--input_options=''{\"num_records\":200000000,\"key_size\":1,\"value_size\":9}'' +--iterations=1 +--fanout=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt new file mode 100644 index 000000000000..a5f6b0934521 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_4 +--iterations=1 +--fanout=4 +--parallelism=16 +--input_options=''{\"num_records\":5000000,\"key_size\":10,\"value_size\":90}'' +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt new file mode 100644 index 000000000000..b261ff58c5bf --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_5 +--iterations=1 +--fanout=8 +--parallelism=16 +--input_options=''{\"num_records\":2500000,\"key_size\":10,\"value_size\":90}'' +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt new file mode 100644 index 000000000000..7e8d1eaa60d9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_gbk_6 +--iterations=4 +--fanout=1 +--parallelism=5 +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":200,\"hot_key_fraction\":1}'' +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file From c5b75ed496f30ec3c95528b29387013e2f5b9cca Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 29 Sep 2023 10:53:08 -0400 Subject: [PATCH 38/64] Create documentation page for Python SDK unrecoverable errors (#28702) * Create documentation page for Python SDK unrecoverable errors * Trailing whitespace * Add link to page on Python SDK page * Add to sidebar * Ditch table for text * Address comments * Update website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md Co-authored-by: tvalentyn * anchor links + edit * last anchor * remove anchor links (didn't render in the sidebar correctly) --------- Co-authored-by: tvalentyn --- .../sdks/python-unrecoverable-errors.md | 61 +++++++++++++++++++ .../content/en/documentation/sdks/python.md | 4 ++ .../partials/section-menu/en/sdks.html | 1 + 3 files changed, 66 insertions(+) create mode 100644 website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md diff --git a/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md b/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md new file mode 100644 index 000000000000..4e5d94ce8a8d --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md @@ -0,0 +1,61 @@ +--- +type: languages +title: "Unrecoverable Errors in Beam Python" +--- + + +# Unrecoverable Errors in Beam Python + +## What is an Unrecoverable Error? + +An unrecoverable error is an issue at job start-up time that will +prevent a job from ever running successfully, usually due to some kind +of misconfiguration. Solving these issues when they occur is key to +successfully running a Beam Python pipeline. + +## Common Unrecoverable Errors + +### Job Submission/Runtime Python Version Mismatch + +If the Python version used for job submission does not match the +Python version used to build the worker container, the job will not +execute. Ensure that the Python version being used for job submission +and the container Python version match. + +### PIP Dependency Resolution Failures + +During worker start-up, dependencies are checked and installed in +the worker container before accepting work. If a pipeline requires +additional dependencies not already present in the runtime environment, +they are installed here. If there’s an issue during this process +(e.g. a dependency version cannot be found, or a worker cannot +connect to PyPI) the worker will fail and may try to restart +depending on the runner. Ensure that dependency versions provided in +your requirements.txt file exist and can be installed locally before +submitting jobs. + +### Dependency Version Mismatches + +When additional dependencies like `torch`, `transformers`, etc. are not +specified via a requirements_file or preinstalled in a custom container +then the worker might fail to deserialize (unpickle) the user code. +This can result in `ModuleNotFound` errors. If dependencies are installed +but their versions don't match the versions in submission environment, +pipeline might have `AttributeError` messages. + +Ensure that the required dependencies at runtime and in the submission +environment are the same along with their versions. For better visibility, +debug logs are added specifying the dependencies at both stages starting in +Beam 2.52.0. For more information, see: https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/#control-dependencies \ No newline at end of file diff --git a/website/www/site/content/en/documentation/sdks/python.md b/website/www/site/content/en/documentation/sdks/python.md index 80c26c258d5b..dc9f6a54d893 100644 --- a/website/www/site/content/en/documentation/sdks/python.md +++ b/website/www/site/content/en/documentation/sdks/python.md @@ -59,3 +59,7 @@ see [Machine Learning](/documentation/sdks/python-machine-learning). ## Python multi-language pipelines quickstart Apache Beam lets you combine transforms written in any supported SDK language and use them in one multi-language pipeline. To learn how to create a multi-language pipeline using the Python SDK, see the [Python multi-language pipelines quickstart](/documentation/sdks/python-multi-language-pipelines). + +## Unrecoverable Errors in Beam Python + +Some common errors can occur during worker start-up and prevent jobs from starting. To learn about these errors and how to troubleshoot them in the Python SDK, see [Unrecoverable Errors in Beam Python](/documentation/sdks/python-unrecoverable-errors). \ No newline at end of file diff --git a/website/www/site/layouts/partials/section-menu/en/sdks.html b/website/www/site/layouts/partials/section-menu/en/sdks.html index 434bddb936b6..73bea15a28d1 100644 --- a/website/www/site/layouts/partials/section-menu/en/sdks.html +++ b/website/www/site/layouts/partials/section-menu/en/sdks.html @@ -43,6 +43,7 @@

  • Machine Learning
  • Managing pipeline dependencies
  • Python multi-language pipelines quickstart
  • +
  • Python Unrecoverable Errors
  • From 5d4c524cf3e590a9f608e4adf146273ae28d81a5 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 29 Sep 2023 18:53:41 +0400 Subject: [PATCH 39/64] Add LoadTests Go ParDo Flink Batch workflow (#28723) * Add LoadTests Go ParDo Flink Batch workflow * Refactoring --- .../beam_LoadTests_Go_ParDo_Flink_Batch.yml | 139 ++++++++++++++++++ .../go_ParDo_Flink_Batch_100_counters.txt | 27 ++++ .../go_ParDo_Flink_Batch_10_counters.txt | 27 ++++ .../go_ParDo_Flink_Batch_10_times.txt | 27 ++++ .../go_ParDo_Flink_Batch_200_times.txt | 27 ++++ 5 files changed, 247 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml new file mode 100644 index 000000000000..34211f9270ff --- /dev/null +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -0,0 +1,139 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Go ParDo Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 2 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-go-pardo-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-go-pardo-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Go_ParDo_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Go ParDo Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Go_ParDo_Flink_Batch"] + job_phrase: ["Run Load Tests Go ParDo Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt + arguments: | + --job_name=load-tests-go-flink-batch-pardo-$(date '+%m%d%H%M%S' --utc) + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Flink Batch Go Load Test 1 (10 times) + timeout-minutes: 180 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=pardo \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_ParDo_Flink_Batch_test_arguments_1 }}' \ + - name: run ParDo Flink Batch Go Load Test 2 (200 times) + timeout-minutes: 180 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=pardo \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_ParDo_Flink_Batch_test_arguments_2 }}' \ + - name: run ParDo Flink Batch Go Load Test 3 (10 counters) + timeout-minutes: 180 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=pardo \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_ParDo_Flink_Batch_test_arguments_3 }}' \ + - name: run ParDo Flink Batch Go Load Test 4 (100 counters) + timeout-minutes: 180 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=pardo \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_ParDo_Flink_Batch_test_arguments_4 }}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt new file mode 100644 index 000000000000..1b60c5380609 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--influx_measurement=go_batch_pardo_4 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt new file mode 100644 index 000000000000..7179f2d469f2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--influx_measurement=go_batch_pardo_3 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90}'' +--iterations=1 +--number_of_counter_operations=10 +--number_of_counters=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt new file mode 100644 index 000000000000..04821e3b6c9a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--influx_measurement=go_batch_pardo_1 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90}'' +--iterations=10 +--number_of_counter_operations=0 +--number_of_counters=0 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt new file mode 100644 index 000000000000..a2606f097f4e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--influx_measurement=go_batch_pardo_2 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90}'' +--iterations=200 +--number_of_counter_operations=0 +--number_of_counters=0 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file From e8342b00ef2f5dc4b7c987c64c8e1d9e164fb7fc Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 29 Sep 2023 18:54:14 +0400 Subject: [PATCH 40/64] Add Load Tests Combine Flink Batch Go workflow (#28661) * Add Load Tests Combine Flink Batch Go workflow * Change cron * Refactoring * Refactoring --- .../beam_LoadTests_Go_Combine_Flink_Batch.yml | 134 ++++++++++++++++++ .../go_Combine_Flink_Batch_10b.txt | 26 ++++ .../go_Combine_Flink_Batch_Fanout_4.txt | 26 ++++ .../go_Combine_Flink_Batch_Fanout_8.txt | 26 ++++ 4 files changed, 212 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt create mode 100644 .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml new file mode 100644 index 000000000000..0226b003b58e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -0,0 +1,134 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Go Combine Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-go-combine-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-go-combine-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Go_Combine_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Go Combine Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Go_Combine_Flink_Batch"] + job_phrase: ["Run Load Tests Go Combine Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt + arguments: | + --job_name=load-tests-go-flink-batch-combine-$(date '+%m%d%H%M%S' --utc) + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Combine Flink Batch Go Load Test 1 (10b records) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=combine \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Flink_Batch_test_arguments_1 }}' \ + - name: Restart Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh restart + - name: run Combine Flink Batch Go Load Test 2 (fanout 4) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=combine \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Flink_Batch_test_arguments_2 }}' \ + - name: run Combine Flink Batch Go Load Test 3 (fanout 8) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=combine \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Flink_Batch_test_arguments_3 }}' + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt new file mode 100644 index 000000000000..6e13d4e185cb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_combine_1 +--input_options=''{\"num_records\":200000000,\"key_size\":1,\"value_size\":9}'' +--fanout=1 +--top_count=20 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt new file mode 100644 index 000000000000..154076a63980 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_combine_2 +--input_options=''{\"num_records\":5000000,\"key_size\":10,\"value_size\":90}'' +--fanout=4 +--top_count=20 +--parallelism=16 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt new file mode 100644 index 000000000000..05e4dccf74de --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_combine_3 +--fanout=8 +--top_count=20 +--parallelism=16 +--input_options=''{\"num_records\":2500000,\"key_size\":10,\"value_size\":90}'' +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file From eae46458d922e9eb614d1c135053adc9b4f4b3bb Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 29 Sep 2023 20:57:15 +0600 Subject: [PATCH 41/64] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch and Cron Fix for other workflows (#28737) * Java_GBK_SparkStructuredStreaming_Batch and cron * description fix * description fix --- ...ava_GBK_SparkStructuredStreaming_Batch.yml | 141 ++++++++++++++++++ .../workflows/beam_PerformanceTests_Cdap.yml | 2 +- .../beam_PerformanceTests_HadoopFormat.yml | 2 +- .../beam_PerformanceTests_MongoDBIO_IT.yml | 2 +- ...beam_PerformanceTests_SparkReceiver_IO.yml | 2 +- ...redStreaming_Batch_2GB_of_100B_records.txt | 25 ++++ ...edStreaming_Batch_2GB_of_100kB_records.txt | 25 ++++ ...uredStreaming_Batch_2GB_of_10B_records.txt | 25 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 25 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 25 ++++ ...ng_Batch_reiterate_4_times_10kB_values.txt | 25 ++++ ...ing_Batch_reiterate_4_times_2MB_values.txt | 25 ++++ 12 files changed, 320 insertions(+), 4 deletions(-) create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml new file mode 100644 index 000000000000..039a9fc3b752 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK SparkStructuredStreaming Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 10 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java GBK SparkStructuredStreaming Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch"] + job_phrase: ["Run Load Tests Java GBK SparkStructuredStreaming Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index 0dfd095ecf5f..f45419a1223f 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '13 4/16 * * *' + - cron: '13 4/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index c80db8a86a16..d73c6f6fb5f5 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '16 7/19 * * *' + - cron: '16 7/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 299d9ac0b77c..0ad21f99f8d4 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '14 5/17 * * *' + - cron: '14 5/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index f2f4dc85ec5e..58c561f6ef64 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '15 6/18 * * *' + - cron: '15 6/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..244483d336f8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..e0a51802e708 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..1003f5ad4cf1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..424d44570877 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..f07fc70a9e0f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..ecbe7dd534a2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..ace93b3a5b07 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file From 65de726ce3647389ea77c7016b955ce75c7315ef Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Fri, 29 Sep 2023 16:59:32 +0200 Subject: [PATCH 42/64] Beam post commit java influx db io it (#28530) * beam_PostCommit_Java_InfluxDbIO_IT and add k8s access action * readme * fix small issues and add namespace sanitization to k8s action --------- Co-authored-by: Andrey Devyatkin --- .github/actions/setup-k8s-access/action.yml | 2 +- .github/workflows/README.md | 4 +- .../beam_PostCommit_Java_InfluxDbIO_IT.yml | 96 +++++++++++++++++++ 3 files changed, 100 insertions(+), 2 deletions(-) create mode 100644 .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml diff --git a/.github/actions/setup-k8s-access/action.yml b/.github/actions/setup-k8s-access/action.yml index 79a2ac8d243c..5248fff429a0 100644 --- a/.github/actions/setup-k8s-access/action.yml +++ b/.github/actions/setup-k8s-access/action.yml @@ -70,4 +70,4 @@ runs: main: echo "Post Cleanup" post: | echo "Post Cleanup" - kubectl delete namespace ${{ steps.replace_namespace.outputs.TEST_NAMESPACE }} \ No newline at end of file + kubectl delete namespace ${{ steps.replace_namespace.outputs.TEST_NAMESPACE }} diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 03479e5c4c3b..0858af5aeac5 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -180,8 +180,10 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex ```Run Python PreCommit (3.8)``` | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ Load Tests GBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) +| [ Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) +| [ Load Tests GBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | [ Load Tests CoGBK Dataflow Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) +| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | [ Load Tests FnApiRunner Microbenchmark Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A |`Run Python Load Tests FnApiRunner Microbenchmark`| [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | [ Load Tests ParDo Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Go ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml new file mode 100644 index 000000000000..f048fdc6f1f7 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.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: Java InfluxDbIO Integration Test + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +jobs: + beam_PostCommit_Java_InfluxDbIO_IT: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + matrix: + job_name: [beam_PostCommit_Java_InfluxDbIO_IT] + job_phrase: [Run Java InfluxDbIO_IT] + if: | + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java InfluxDbIO_IT' + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install InfluxDB + id: install_influxdb + run: | + kubectl apply -f ${{ github.workspace }}/.test-infra/kubernetes/influxdb/influxdb.yml + kubectl wait svc/influxdb-load-balancer-service --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' + loadbalancer_IP=$(kubectl get svc influxdb-load-balancer-service -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo influxdb_IP=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Run Java InfluxDbIO_IT + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:influxdb:integrationTest --tests org.apache.beam.sdk.io.influxdb.InfluxDbIOIT + arguments: --info -DintegrationTestRunner=direct -DintegrationTestPipelineOptions='["--influxDBURL=http://${{ steps.install_influxdb.outputs.influxdb_IP }}:8086","--influxDBUserName=superadmin","--influxDBPassword=supersecretpassword","--databaseName=db1"]' From 75d7c70a9a05a112d1df8975fef0b190afc1682d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 29 Sep 2023 11:24:23 -0400 Subject: [PATCH 43/64] Fix Jpms test (#28726) * Fix Jpms test * Configure main src also compile in specified Java ver * Fix Java11 case --- sdks/java/testing/jpms-tests/build.gradle | 40 +++++++++-------------- 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index f781c29b8480..6321f874c903 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -21,7 +21,14 @@ import groovy.json.JsonOutput plugins { id 'org.apache.beam.module' } -javaVersion="1.11" + +// overwrite javaVersion before applyJavaNature +if (project.hasProperty("compileAndRunTestsWithJava17")) { + javaVersion = '1.17' +} else { + javaVersion = '1.11' +} + applyJavaNature( exportJavadoc: false, publish: false, @@ -33,30 +40,15 @@ enableJavaPerformanceTesting() description = "Apache Beam :: SDKs :: Java :: Testing :: JPMS Tests" ext.summary = "E2E test for Java 9 modules" -// Java 17 needs compileJava to add-exports and add-opens for error prone -if (project.hasProperty("compileAndRunTestsWithJava17")) { - def java17Home = project.findProperty("java17Home") - project.tasks.withType(JavaCompile) { +// direct compileJava to use specified java version. +project.tasks.compileJava { + if (project.hasProperty("compileAndRunTestsWithJava11")) { options.fork = true - options.forkOptions.javaHome = java17Home as File - options.compilerArgs += ['-Xlint:-path'] - options.compilerArgs.addAll(['--release', '17']) - // Error prone requires some packages to be exported/opened for Java 17 - // Disabling checks since this property is only used for Jenkins tests - // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support - options.errorprone.errorproneArgs.add("-XepDisableAllChecks") - options.forkOptions.jvmArgs += [ - "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" - ] + options.forkOptions.javaHome = project.findProperty("java11Home") as File + } else if (project.hasProperty("compileAndRunTestsWithJava17")) { + options.fork = true + options.forkOptions.javaHome = project.findProperty("java17Home") as File + setJava17Options(options) } } From 8a31ffe09ba2f7d664a5e44f485d7cf5cebb468a Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Fri, 29 Sep 2023 17:45:14 +0200 Subject: [PATCH 44/64] replace comment.body with comment.id (#28739) --- .github/workflows/README.md | 2 +- .github/workflows/beam_CancelStaleDataflowJobs.yml | 2 +- .github/workflows/beam_CleanUpGCPResources.yml | 2 +- .github/workflows/beam_CleanUpPrebuiltSDKImages.yml | 2 +- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 +- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PostCommit_Python_Arm.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...am_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 +- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Website_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Website_Test.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_Go.yml | 2 +- .github/workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .github/workflows/beam_PreCommit_ItFramework.yml | 2 +- .github/workflows/beam_PreCommit_Java.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../beam_PreCommit_Java_File-schema-transform_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Flink_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_IOs_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Spark3_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Kotlin_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Portable_Python.yml | 2 +- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocs.yml | 2 +- .github/workflows/beam_PreCommit_PythonFormatter.yml | 2 +- .github/workflows/beam_PreCommit_PythonLint.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 2 +- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Python_Integration.yml | 2 +- .github/workflows/beam_PreCommit_Python_PVR_Flink.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .github/workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_RAT.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .github/workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .github/workflows/beam_PreCommit_Website_Stage_GCS.yml | 2 +- .github/workflows/beam_PreCommit_Whitespace.yml | 2 +- .../workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- .github/workflows/build_wheels.yml | 2 +- .github/workflows/dask_runner_tests.yml | 2 +- .github/workflows/go_tests.yml | 2 +- .github/workflows/java_tests.yml | 2 +- .github/workflows/local_env_tests.yml | 2 +- .github/workflows/playground_frontend_test.yml | 2 +- .github/workflows/python_dependency_tests.yml | 2 +- .github/workflows/python_tests.yml | 2 +- .github/workflows/tour_of_beam_backend.yml | 2 +- .github/workflows/tour_of_beam_backend_integration.yml | 2 +- .github/workflows/tour_of_beam_frontend_test.yml | 2 +- .github/workflows/typescript_tests.yml | 2 +- 173 files changed, 173 insertions(+), 173 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 0858af5aeac5..8fba73ed8fc3 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -119,7 +119,7 @@ Concurrency groups are a way of making sure that no more than one Actions run is ``` concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true ``` diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index df896528c84e..226ee78f01aa 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 7a420822f9df..42aeccb1a3b0 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index 3ecf192aded9..62197137ca0c 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index b21428413c01..0ea3207b505a 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -44,7 +44,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index 8e5651e29279..2473a44b010b 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -44,7 +44,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 4e647962ccac..90e107ee0c17 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 0fc0f8d400de..07cd627059b2 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index d2d3be47e45d..3eed85bc2026 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index cf3064a6a7de..1bd828d08ee0 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 02896bd3a383..55e4f99afc74 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 56b1e6dc91bb..7ca2f57ce78d 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index c4e37fd495bf..3b3b51b94f25 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 22d605ed1c2d..939d2646d352 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -37,7 +37,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index f50ae9d30f6c..a998e0242254 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index af01933be446..0c87a81c5bed 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 0dfea680f824..2526f9b56531 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index b8014342042c..6fd1150aecd1 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index 3aec68316f81..b123134cd239 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index fd5c60952b7b..fa28cdf402bf 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index da4606400e8b..f9c7175ccd21 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 6c35a10e5e92..3ee009747a88 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 5d0d2273cf55..398be301eecf 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 4e26f29276b2..44b89acb642e 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 0a4801d46c6c..dbeb84ab660b 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index aa5e3b40c9dd..f4ba8ada32ba 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index a81df741d9cc..9758fda66eb7 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index ef90fbad5bf0..74b786b599e1 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 3eb93e6687f8..3b7836990b69 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 06438510400b..a0957f643279 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 2386d7e26f38..4daa13da8b13 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index 9123c9079605..a03c447416f3 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 7492eb9b8262..a43e7e6d311f 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 709ec8215983..991e4f71b1c4 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 06b88ee5bcb8..041c031f3f76 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index e87c62674c40..2a375d71981a 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index ce4e17dc25ea..8d54c7707258 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index dc719c2d53dd..7f914dc35880 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 6a458e174376..e19831c60732 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index e062b108aef6..fd35fe4ec776 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 6dc6cbd5f1a0..2284e7fa06bf 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 92ddeeb8663e..596ef873c964 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 37ec6b145f35..19c2f3f8cb16 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 708ef0b7ad16..013b34bc807b 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index aa665508ddee..3a9e0140f818 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index ca380efa5cb4..edf9a04f22c3 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 24e2c8014336..79447f610809 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 168cd245e0db..8e5dbac0c4cd 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 20aac8a30608..2ff883dafa75 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index d5de5fbfa3ff..c0339100845e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 0613f794a4f4..97e35490c25d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 146a88c921d2..5800c338ed16 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index fcaa3c6eabd4..08504316333a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index e6fe19393d13..1cc0193b1e69 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 89364f32865b..21bacac11d59 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index e5e397c8c9d3..75c07bc49783 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 240b0e43d271..7185f588f463 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 38c68059bdbe..3f682ea57dde 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 4cb99f85f0fb..f9884d651978 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 0a5758ac9366..a7a214c7c5a9 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 74eb8e8290d8..a77c4e96dc51 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 14cdecb356bd..40c508b38e6e 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 7c792d7a3c27..85d766b0575c 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index ccf03918f29d..17112cf18e2a 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 073ed0aeda64..ccc7a998df27 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index be8f0e10dc18..0fb2302b8ed1 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index 81c9b4a8b484..db95a48a3007 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index ca5753010d1c..fb7102a5b52e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index ded9ff0a4bd5..a95682e415d4 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 4119ddd56020..572f477773b6 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index 608bba248b3b..a1652f3a18de 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index bd0bbe1d6ff1..c41605391d40 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 6fda3a210aaf..d42835727d36 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 34364bb38bcc..31025df81ddc 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index f753b3cf15df..d17d15029c2c 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 6cc132935e66..386d91518795 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index d27c5718d6f9..a753cde6e56d 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 465dc9966023..2153e9908898 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 1d8b35098e45..9387e9f74831 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index 4cb99532543d..a1e23b530a26 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index b186e8ffcd81..cd06181c456d 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 988b153986e9..eaee2b669873 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 0ab819c57a02..67effa70b8b5 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 687c77e572f2..2f8a2eb8a3da 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 71b3d27473e1..386fa304606d 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index ea3fd2cb86b8..2432ddc30e2a 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index d575bbeabbc8..bebac78a6315 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index f808456b8c97..8cf16cfe225d 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 8ca97bb23edb..193526ffe7f3 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index 7a93873168e0..f044b154c0ab 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index 227f3c7648ab..b7b561d54c1f 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 8156df15133c..1c40a3c8d129 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 1a669c157007..0e60579ec64d 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index e8ec1287be51..6161f1422566 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -36,7 +36,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index ab5c17c55f4c..9a58f42fef78 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -122,7 +122,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 220dea3208cc..2d27ecc38be0 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index 03ee673e2e24..c812aac4d447 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index fede8fa6d548..8898cf36f188 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 7e20dd043cad..84ffe4dab834 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 5f63a25440b8..35f0e061f091 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index caa1c475f5b0..b23490afe8f7 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -54,7 +54,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 8f2a5cde3749..0dd75cb0a6d2 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 1c304500b567..17e1876d20f3 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index db348a7684af..6f10c210b010 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 27a3e175e7e1..6775708c2b8f 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 1dfd9ea1eb4d..29a637baff3a 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 3fadb5e7de0f..03f4d32861b1 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index 9d4b798c4f38..cf8cefe388f3 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -60,7 +60,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index d256bca9ebaa..1bfd58d6a0d6 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index e4a04839cef4..cefbf897de85 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -53,7 +53,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index b7720682ce23..7c821a024742 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 5734e0e9d453..23d0afb933de 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 6230c8dbae8a..777725d7c3ec 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index ad69f5fb0895..2b93fa4c524b 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -76,7 +76,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index b20e9886e701..2825719a24ef 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -51,7 +51,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 9acbe5f8705a..6d6bd34deef2 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 869c189c1c9d..2d36c05d2881 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 160e01520845..fa8d2721d38e 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 64fc4122923b..629190e328e5 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -58,7 +58,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index d1788288d3d5..47411e0df379 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 693cead4b031..40fdb06d3556 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 472b0f7f5531..f210affbb21f 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index ad0e5ac9feda..6f30a3dd84ba 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index b6f51b243a8f..8e8041d434c7 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 9679b1825cf5..b592d704661d 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -40,7 +40,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 30dd710e1848..327f65436560 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -60,7 +60,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index f2a27da20dd2..c00d0e43b619 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index ed7c55c98f4d..b0b73d944275 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 962c1526c90a..a259c09beb74 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index efd8d472f8ae..cd2741ead81a 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index ad64ff286aab..9c370e607368 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index f8eaec4c11c4..3e27d5c5cfe8 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -54,7 +54,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index 8b834682ffd5..c7d13e49f2a4 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 2bb53629614d..9b2ada1da776 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -36,7 +36,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index ff0a31eb0d0e..7e453c379281 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 649e4ad28c65..71528a884312 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 42fe14cd8338..7d26f12ee635 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index b30163b12a47..fb6c0f7696aa 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index 35e2c535f4ea..0134c497b836 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -66,7 +66,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 80c69afce6e2..c891a79cefd0 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 669f316549c5..aa119cf674c2 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index a67e8afa3a4e..844d41f2cb52 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 1a3335b370e9..23093e1db006 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 9d290f1ba86d..8a4558c8988d 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 65002f9da894..33be9644d34a 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index a7a0ec1836ce..2862d7d5936c 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 1b03b4c0a35a..7f980885180a 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 5a1b7bc720f3..5b377f23774e 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index a0011354749a..c268b4ed78f7 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index 775af7f39d24..b0c5ab4fa34a 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 291dcde8665a..88ac59c9de96 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_RAT.yml b/.github/workflows/beam_PreCommit_RAT.yml index 390413fb7aca..c129fc12114f 100644 --- a/.github/workflows/beam_PreCommit_RAT.yml +++ b/.github/workflows/beam_PreCommit_RAT.yml @@ -45,7 +45,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 5a3135562e26..ee9799346f1f 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 29a89d7f54a8..f1c733418b8d 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 468c821bec14..7547bd396815 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index 552a92e104a1..394f627adbab 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -43,7 +43,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index 21c760f2e525..a2df0033f98b 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -33,7 +33,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index 87218dc28033..4ae4e1a99a0c 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index f910f9a88da3..00cda54912ac 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 03a976cfe444..065c5cc7fd8f 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index d4c9178e6c91..9a4ff4144ac5 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -44,7 +44,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 1028dd79af02..94248be0c008 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -33,7 +33,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 46d2707ca063..423a304db825 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -29,7 +29,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 32b494da25be..6884dd692522 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -32,7 +32,7 @@ on: workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: build: diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index ceff29b50d4f..1132ba1c196b 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -37,7 +37,7 @@ on: 'examples/kotlin/**', 'release/**', 'buildSrc/**'] # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} diff --git a/.github/workflows/local_env_tests.yml b/.github/workflows/local_env_tests.yml index 32e2975c0712..7dbff4feeb45 100644 --- a/.github/workflows/local_env_tests.yml +++ b/.github/workflows/local_env_tests.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/playground_frontend_test.yml b/.github/workflows/playground_frontend_test.yml index 6f6e02a9697c..4d70197a11f7 100644 --- a/.github/workflows/playground_frontend_test.yml +++ b/.github/workflows/playground_frontend_test.yml @@ -28,7 +28,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index 0c91e64b0db9..6fd865bda754 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -11,7 +11,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 406949eda96e..57ec895c2431 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -37,7 +37,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: diff --git a/.github/workflows/tour_of_beam_backend.yml b/.github/workflows/tour_of_beam_backend.yml index 5c67c9f54fe3..3585bd0e670f 100644 --- a/.github/workflows/tour_of_beam_backend.yml +++ b/.github/workflows/tour_of_beam_backend.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: diff --git a/.github/workflows/tour_of_beam_backend_integration.yml b/.github/workflows/tour_of_beam_backend_integration.yml index 8f56d3f2e2fa..c96611b3215b 100644 --- a/.github/workflows/tour_of_beam_backend_integration.yml +++ b/.github/workflows/tour_of_beam_backend_integration.yml @@ -37,7 +37,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/tour_of_beam_frontend_test.yml b/.github/workflows/tour_of_beam_frontend_test.yml index 5337bb7dd720..49bc9ef357d6 100644 --- a/.github/workflows/tour_of_beam_frontend_test.yml +++ b/.github/workflows/tour_of_beam_frontend_test.yml @@ -30,7 +30,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true env: diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index 825b2808af5b..a4aa14c42efd 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -33,7 +33,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' cancel-in-progress: true jobs: typescript_unit_tests: From 4204ba34a5df33a3016d02ac1558258773a094d7 Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Fri, 29 Sep 2023 17:45:49 +0200 Subject: [PATCH 45/64] postcommit java singlestoreIO_IO (#28741) * postcommit java singlestoreIO_IO * bump singlestore operator image --- .../beam_PostCommit_Java_SingleStoreIO_IT.yml | 105 ++++++++++++++++++ .../kubernetes/singlestore/sdb-operator.yaml | 2 +- 2 files changed, 106 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml new file mode 100644 index 000000000000..40a1dc8faa64 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -0,0 +1,105 @@ +# 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 Java SingleStoreIO IT + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */23 * * *' + workflow_dispatch: +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Java_SingleStoreIO_IT: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + fail-fast: false + matrix: + job_name: [beam_PostCommit_Java_SingleStoreIO_IT] + job_phrase: [Run Java SingleStoreIO_IT] + if: | + github.event_name == 'push' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java SingleStoreIO_IT' + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Singlestore operator + run: | + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-rbac.yaml + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-cluster-crd.yaml + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-operator.yaml + kubectl wait --for=condition=Ready pod -l name=sdb-operator --timeout=120s + - name: Install Singlestore cluster + id: install_singlestore + run: | + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-cluster.yaml + kubectl wait --for=jsonpath='{.status.phase}'=Running memsqlclusters.memsql.com --all --timeout=120s + kubectl wait svc/svc-sdb-cluster-ddl --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + loadbalancer_IP=$(kubectl get svc svc-sdb-cluster-ddl -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo lb_ip=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Run Java SingleStore IO IT + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIODefaultMapperIT + arguments: --info -DintegrationTestRunner=dataflow -DintegrationTestPipelineOptions='["--tempRoot=gs://temp-storage-for-perf-tests","--project=apache-beam-testing","--runner=DataflowRunner","--singleStoreUsername=admin","--singleStorePassword=secretpass","--singleStorePort=3306","--numberOfRecords=1000", "--singleStoreServerName=${{ steps.install_singlestore.outputs.lb_ip }}"]' diff --git a/.test-infra/kubernetes/singlestore/sdb-operator.yaml b/.test-infra/kubernetes/singlestore/sdb-operator.yaml index 5c6bb1f76e13..fc2b2585b24a 100644 --- a/.test-infra/kubernetes/singlestore/sdb-operator.yaml +++ b/.test-infra/kubernetes/singlestore/sdb-operator.yaml @@ -32,7 +32,7 @@ spec: serviceAccountName: sdb-operator containers: - name: sdb-operator - image: singlestore/operator:3.0.32-db8f5aff + image: singlestore/operator:3.0.98-156a0090 imagePullPolicy: Always args: [ # Cause the operator to merge rather than replace annotations on services From 0a6b29da878f56d3adb0ce166407296229722e1e Mon Sep 17 00:00:00 2001 From: Steven van Rossum Date: Fri, 29 Sep 2023 18:33:12 +0200 Subject: [PATCH 46/64] Add missing access modifiers to SimpleRateLimitPolicy (#28389) --- .../java/org/apache/beam/sdk/io/googleads/GoogleAdsV14.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV14.java b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV14.java index d4f2119e3f20..46448e0110d9 100644 --- a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV14.java +++ b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV14.java @@ -645,11 +645,11 @@ void onBeforeRequest(String developerToken, String customerId, Message request) public static class SimpleRateLimitPolicy implements RateLimitPolicy { private final RateLimiter rateLimiter; - SimpleRateLimitPolicy(double permitsPerSecond) { + public SimpleRateLimitPolicy(double permitsPerSecond) { rateLimiter = RateLimiter.create(permitsPerSecond); } - SimpleRateLimitPolicy(double permitsPerSecond, long warmupPeriod, TimeUnit unit) { + public SimpleRateLimitPolicy(double permitsPerSecond, long warmupPeriod, TimeUnit unit) { rateLimiter = RateLimiter.create(permitsPerSecond, warmupPeriod, unit); } From b10546cee258d350d2375847b13b3c7f1d9cad3b Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 29 Sep 2023 21:14:05 +0400 Subject: [PATCH 47/64] Add Load Tests SideInput Flink Batch Go workflow (#28662) * Add Load Tests SideInput Flink Batch Go workflow * Refactoring * Refactoring --- ...eam_LoadTests_Go_SideInput_Flink_Batch.yml | 119 ++++++++++++++++++ ...o_SideInput_Flink_Batch_First_Iterable.txt | 25 ++++ .../go_SideInput_Flink_Batch_Iterable.txt | 24 ++++ 3 files changed, 168 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml new file mode 100644 index 000000000000..8600c5cd3717 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -0,0 +1,119 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Go SideInput Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 23 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-go-sideinput-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-go-sideinput-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Go_SideInput_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Go SideInput Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Go_SideInput_Flink_Batch"] + job_phrase: ["Run Load Tests Go SideInput Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt + arguments: | + --job_name=load-tests-go-flink-batch-sideinput-$(date '+%m%d%H%M%S' --utc) + - name: Start Flink with parallelism 10 + env: + FLINK_NUM_WORKERS: 10 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run SideInput Flink Batch Go Load Test 1 (first iterable) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=sideinput \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_SideInput_Flink_Batch_test_arguments_1 }}' \ + - name: run SideInput Flink Batch Go Load Test 2 (iterable) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=sideinput \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_SideInput_Flink_Batch_test_arguments_2 }}' + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt b/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt new file mode 100644 index 000000000000..0d50916b8bd6 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--influx_namespace=flink +--influx_measurement=go_batch_sideinput_1 +--input_options=''{\"num_records\":400000,\"key_size\":100,\"value_size\":900}'' +--access_percentage=1 +--parallelism=10 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt b/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt new file mode 100644 index 000000000000..ca11bc55faa9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt @@ -0,0 +1,24 @@ +# 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. + +--influx_namespace=flink +--influx_measurement=go_batch_sideinput_2 +--input_options=''{\"num_records\":400000,\"key_size\":100,\"value_size\":900}'' +--parallelism=10 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file From 5d6dceeba0114242596e89f2da618d4400997fb2 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 29 Sep 2023 21:14:13 +0400 Subject: [PATCH 48/64] Add Load Tests CoGBK Flink Batch Go workflow (#28660) * Add Load Tests CoGBK Flink Batch Go workflow * Change cron * Refactoring * Refactoring --- .../beam_LoadTests_Go_CoGBK_Flink_batch.yml | 129 ++++++++++++++++++ .../go_CoGBK_Flink_Batch_MultipleKey.txt | 26 ++++ .../go_CoGBK_Flink_Batch_Reiteration_10KB.txt | 26 ++++ .../go_CoGBK_Flink_Batch_Reiteration_2MB.txt | 26 ++++ 4 files changed, 207 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml create mode 100644 .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt create mode 100644 .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt create mode 100644 .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml new file mode 100644 index 000000000000..cf355a2ee98d --- /dev/null +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -0,0 +1,129 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Go CoGBK Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 14 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-go-cogbk-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-go-cogbk-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Go_CoGBK_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Go CoGBK Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Go_CoGBK_Flink_Batch"] + job_phrase: ["Run Load Tests Go CoGBK Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt + arguments: | + --job_name=load-tests-go-flink-batch-cogbk-$(date '+%m%d%H%M%S' --utc) + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK Flink Batch Go Load Test 1 (multiple keys) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=cogbk \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_CoGBK_Flink_Batch_test_arguments_1 }}' \ + - name: run CoGBK Flink Batch Go Load Test 2 (reiterate 4 times 10KB values) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=cogbk \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_CoGBK_Flink_Batch_test_arguments_2 }}' \ + - name: run CoGBK Flink Batch Go Load Test 3 (reiterate 4 times 2MB values) + timeout-minutes: 240 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:go:test:load:run + arguments: | + -PloadTest.mainClass=cogbk \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_CoGBK_Flink_Batch_test_arguments_3 }}' + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt new file mode 100644 index 000000000000..e508c47bbaa8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_measurement=go_batch_cogbk_1 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":5,\"hot_key_fraction\":1}'' +--co_input_options=''{\"num_records\":2000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":1000,\"hot_key_fraction\":1}'' +--iterations=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt new file mode 100644 index 000000000000..b399b283f300 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_measurement=go_batch_cogbk_2 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":200000,\"hot_key_fraction\":1}'' +--co_input_options=''{\"num_records\":2000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":1000,\"hot_key_fraction\":1}'' +--iterations=4 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt new file mode 100644 index 000000000000..957f0b4c0272 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt @@ -0,0 +1,26 @@ +# 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. + +--influx_measurement=go_batch_cogbk_3 +--influx_namespace=flink +--input_options=''{\"num_records\":20000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":1000,\"hot_key_fraction\":1}'' +--co_input_options=''{\"num_records\":2000000,\"key_size\":10,\"value_size\":90,\"num_hot_keys\":1000,\"hot_key_fraction\":1}'' +--iterations=4 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest +--runner=FlinkRunner \ No newline at end of file From 5e38decf9e723a385057131b01bbd33d8c60bda3 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 29 Sep 2023 14:20:25 -0400 Subject: [PATCH 49/64] Add callout of PIP failures as unrecoverable, link to new documentation (#28743) --- sdks/python/container/piputil.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/python/container/piputil.go b/sdks/python/container/piputil.go index 67488bdc39f7..1bafe422d457 100644 --- a/sdks/python/container/piputil.go +++ b/sdks/python/container/piputil.go @@ -32,6 +32,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/util/execx" ) +const unrecoverableURL string = "https://beam.apache.org/documentation/sdks/python-unrecoverable-errors/index.html#pip-dependency-resolution-failures" + // pipInstallRequirements installs the given requirement, if present. func pipInstallRequirements(ctx context.Context, logger *tools.Logger, files []string, dir, name string) error { pythonVersion, err := expansionx.GetPythonVersion() @@ -56,7 +58,7 @@ func pipInstallRequirements(ctx context.Context, logger *tools.Logger, files []s err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...) if err != nil { bufLogger.FlushAtError(ctx) - return err + return fmt.Errorf("PIP failed to install dependencies, got %s. This error may be unrecoverable, see %s for more information", err, unrecoverableURL) } bufLogger.FlushAtDebug(ctx) return nil @@ -114,7 +116,7 @@ func pipInstallPackage(ctx context.Context, logger *tools.Logger, files []string err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...) if err != nil { bufLogger.FlushAtError(ctx) - return err + return fmt.Errorf("PIP failed to install dependencies, got %s. This error may be unrecoverable, see %s for more information", err, unrecoverableURL) } else { bufLogger.FlushAtDebug(ctx) } @@ -122,7 +124,7 @@ func pipInstallPackage(ctx context.Context, logger *tools.Logger, files []string err = execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...) if err != nil { bufLogger.FlushAtError(ctx) - return err + return fmt.Errorf("PIP failed to install dependencies, got %s. This error may be unrecoverable, see %s for more information", err, unrecoverableURL) } bufLogger.FlushAtDebug(ctx) return nil @@ -133,7 +135,7 @@ func pipInstallPackage(ctx context.Context, logger *tools.Logger, files []string err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...) if err != nil { bufLogger.FlushAtError(ctx) - return err + return fmt.Errorf("PIP failed to install dependencies, got %s. This error may be unrecoverable, see %s for more information", err, unrecoverableURL) } bufLogger.FlushAtDebug(ctx) return nil From 80311628c27ea68704f46d9f5e6edea6466dfcb3 Mon Sep 17 00:00:00 2001 From: Damon Date: Fri, 29 Sep 2023 13:04:18 -0700 Subject: [PATCH 50/64] Remove warning from catch in table exists validation (#28288) * Remove warning from catch in table exists validation * Remove warning from catch in read table exists validation * Throw RuntimeException instead --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 9f3c627a89ef..92a0af205482 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -19,8 +19,8 @@ import static org.apache.beam.sdk.io.gcp.bigtable.BigtableServiceFactory.BigtableServiceEntry; import static org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; @@ -689,14 +689,13 @@ public final String toString() { private void validateTableExists( BigtableConfig config, BigtableReadOptions readOptions, PipelineOptions options) { if (config.getValidate() && config.isDataAccessible() && readOptions.isDataAccessible()) { - String tableId = checkNotNull(readOptions.getTableId().get()); + ValueProvider tableIdProvider = checkArgumentNotNull(readOptions.getTableId()); + String tableId = checkArgumentNotNull(tableIdProvider.get()); try { - checkArgument( - getServiceFactory().checkTableExists(config, options, tableId), - "Table %s does not exist", - tableId); + boolean exists = getServiceFactory().checkTableExists(config, options, tableId); + checkArgument(exists, "Table %s does not exist", tableId); } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + throw new RuntimeException(e); } } } @@ -1122,14 +1121,13 @@ public String toString() { private void validateTableExists( BigtableConfig config, BigtableWriteOptions writeOptions, PipelineOptions options) { if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { - String tableId = checkNotNull(writeOptions.getTableId().get()); + ValueProvider tableIdProvider = checkArgumentNotNull(writeOptions.getTableId()); + String tableId = checkArgumentNotNull(tableIdProvider.get()); try { - checkArgument( - factory.checkTableExists(config, options, writeOptions.getTableId().get()), - "Table %s does not exist", - tableId); + boolean exists = factory.checkTableExists(config, options, tableId); + checkArgument(exists, "Table %s does not exist", tableId); } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + throw new RuntimeException(e); } } } From 01197c42319422b91b6c59e064418ceeefa6c6ec Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 29 Sep 2023 23:31:36 -0400 Subject: [PATCH 51/64] Add GBK, CoGBK, SideInputs and Combine python batch load tests to perf alert tool (#28746) * Add CoGbk, GBK, sideinputs, and Combine test to the perf alert config * Modify test links * Add space --- .../testing/analyzers/tests_config.yaml | 189 ++++++++++++++++++ 1 file changed, 189 insertions(+) diff --git a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml index bc74f292c487..f808f5e41d74 100644 --- a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml +++ b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml @@ -101,3 +101,192 @@ test_cloudml_benchmark_criteo_10GB-runtime_sec: metrics_table: cloudml_benchmark_criteo_10GB project: apache-beam-testing metric_name: runtime_sec + +# Python Combine load tests at http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1 +combine_python_batch_2gb_10_byte_records: + test_description: + Combine Python Load Test 2 GB 10 byte records + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L76C24-L76C65 + Test dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=2 + test_target: apache_beam.testing.load_tests.combine_test + metrics_dataset: load_test + metrics_table: python_dataflow_batch_combine_1 + metric_name: runtime + project: apache-beam-testing + +combine_python_batch_2gb_fanout_4: + test_description: + Combine Python Load test - 2GB Fanout 4 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L52 + Test Dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=4 + test_target: apache_beam.testing.load_tests.combine_test + metrics_dataset: load_test + metrics_table: python_dataflow_batch_combine_4 + metric_name: runtime + project: apache-beam-testing + +combine_python_batch_2gb_fanout_8: + test_description: Combine Python Load test - 2GB Fanout 8 + test_target: apache_beam.testing.load_tests.combine_test + metrics_dataset: load_test + metrics_table: python_dataflow_batch_combine_5 + metric_name: runtime + project: apache-beam-testing + +# Python Batch GBK load tests at http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now +gbk_python_batch_load_test_2gb_of_10B_records: + test_description: + GroupByKey Python Load test - 2GB of 10B records + python | GBK | Small records (10B) + Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=2 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy#L36C25-L36C72 + test_target: apache_beam.testing.load_tests.group_by_key_test + metrics_table: python_dataflow_batch_gbk_1 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +gbk_python_batch_load_test_2gb_of_100B_records: + test_description: + GroupByKey Python Load test - 2GB of 100B records + python | GBK | Medium records (100B) + Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=3 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy#L60 + test_target: apache_beam.testing.load_tests.group_by_key_test + metrics_table: python_dataflow_batch_gbk_2 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +gbk_python_batch_load_test_2gb_of_100KB_records: + test_description: + GroupByKey Python Load test - 2GB of 100kB records + python | GBK | Large records (100kB) + Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-6M&to=now&viewPanel=4&inspect=4 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy#L83 + test_target: apache_beam.testing.load_tests.group_by_key_test + metrics_table: python_dataflow_batch_gbk_3 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +gbk_python_batch_load_test_fanout_4_times_with_2GB_10byte_records_total: +# this test looks little noisy. Remove this if it causes too many false +# positives. + test_description: + GroupByKey Python Load test - fanout 4 times with 2GB 10-byte records total + python | GBK | Fanout 4 + Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=5 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy#L106 + test_target: apache_beam.testing.load_tests.group_by_key_test + metrics_table: python_dataflow_batch_gbk_4 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + + +gbk_python_batch_load_test_fanout_8_times_with_2GB_10byte_records_total: +# this test looks little noisy. Remove this if it causes too many false +# positives. + test_description: + GroupByKey Python Load test - fanout 8 times with 2GB 10-byte records total + python | GBK | Fanout 8 + Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=6 + Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy#L129 + metrics_table: python_dataflow_batch_gbk_5 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +# Python SideInput load tests at http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now + +sideinpts_python_batch_1gb_1kb_10workers_1000window_1key_percent_dict: + test_description: + python | Side Input | 1 GB dictionary, 1% of keys, 1000 fixed windows + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L120 + Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=8 + metrics_table: python_dataflow_batch_sideinput_7 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + + +sideinpts_python_batch_1gb_1kb_10workers_1000window_99key_percent_dict: + test_description: + python | Side Input | 1 GB dictionary, 99% of keys, 1000 fixed windows + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L133 + Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=9 + metrics_table: python_dataflow_batch_sideinput_8 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: + test_description: + python | Side Input | 10 GB iterable, 1% of elements, 1000 fixed windows + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L146 + Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=10 + metrics_table: python_dataflow_batch_sideinput_9 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + + +sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: + test_description: + python | Side Input | 10 GB iterable, all elements, 1000 fixed windows + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L159 + Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=11 + metrics_table: python_dataflow_batch_sideinput_10 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +# Python CoGBK load tests at http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python + +cogbk_python_batch_load_test_2GB_of_100B_records_with_a_single_key: + test_description: + CoGroupByKey Python Load test - 2GB of 100B records with a single key + python | coGBK | 100B records with a single key + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L32C25-L32C76 + Test Dashboard - http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&viewPanel=2 + test_target: apache_beam.testing.load_tests.co_group_by_key_test + metrics_table: python_dataflow_batch_cogbk_1 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +cogbk_python_batch_load_test_2GB_of_100B_records_with_a_multiple_key: + test_description: + CoGroupByKey Python Load test - 2GB of 100B records with multiple keys + python | coGBK | 100B records with multiple keys + + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L64 + + Test Dashboard - http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&viewPanel=3 + metrics_table: python_dataflow_batch_cogbk_2 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +cogbk_python_batch_load_test_reiterate_4times_10KB_values: + test_description: + CoGroupByKey Python Load test - reiterate 4 times 10kB values + python | coGBK | reiteration 10kB value + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L96 + Test Dashboard - http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&viewPanel=4 + metrics_table: python_dataflow_batch_cogbk_3 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing + +cogbk_python_batch_load_test_reiterate_4times_2MB_values: + test_description: + CoGroupByKey Python Load test - reiterate 4 times 2 MB values + python | coGBK | reiteration 2MB value + Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L128 + Test Dashboard - http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&viewPanel=5 + metrics_table: python_dataflow_batch_cogbk_4 + metrics_dataset: load_test + metric_name: runtime + project: apache-beam-testing \ No newline at end of file From 2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 2 Oct 2023 09:58:49 -0400 Subject: [PATCH 52/64] Restrict build_runner_image to pull_request instead of pull_request_target (#28745) Helps minimize risk of escalation of privileges (we may want to move to a dedicated secret for this in the future) --- .github/workflows/build_runner_image.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_runner_image.yml b/.github/workflows/build_runner_image.yml index 069b8b7db68c..6071d936958a 100644 --- a/.github/workflows/build_runner_image.yml +++ b/.github/workflows/build_runner_image.yml @@ -22,7 +22,7 @@ on: push: branches: ['master'] paths: ['.github/gh-actions-self-hosted-runners/arc/images/**','.github/workflows/build_runner_image.yml'] - pull_request_target: + pull_request: branches: ['master'] paths: ['.github/gh-actions-self-hosted-runners/arc/images/**'] env: From f2194f6b4c82e023a707a8415cb858e968e8c529 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Mon, 2 Oct 2023 14:06:06 +0000 Subject: [PATCH 53/64] Temporarily Ignore 2 FhirIO Integration Tests (#28717) --- .../org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java index e2422f9e172c..fc92a568ba4d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java @@ -43,6 +43,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -126,6 +127,7 @@ public void teardown() throws IOException { } } + @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearch() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); @@ -155,6 +157,7 @@ public void testFhirIOSearch() { pipeline.run().waitUntilFinish(); } + @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearchWithGenericParameters() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); From bf1e82975821c7a8301cf927215c29070868640a Mon Sep 17 00:00:00 2001 From: edman124 Date: Mon, 2 Oct 2023 07:08:55 -0700 Subject: [PATCH 54/64] Report total active threads (#28513) * New active threads metric and initial tests * create unit tests for max active threads metric * remove test filter * fix formatting with spotless apply * revert format changes * revert format change straggler * revert format change straggler * remove unnecessary comment * synchronize threads in unit tests * fix formatting with spotless apply * remove comments and rename counter * fix formatting with spotless apply * fix tests for StreamingDataflowWorker change and fixed createMockWork --- .../worker/DataflowSystemMetrics.java | 2 + .../worker/StreamingDataflowWorker.java | 11 +++ .../worker/util/BoundedQueueExecutor.java | 4 + .../worker/StreamingDataflowWorkerTest.java | 75 ++++++++++++++++++- 4 files changed, 91 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java index e9766fe387e2..ee2a04af9982 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java @@ -40,6 +40,8 @@ public enum StreamingSystemCounterNames { JAVA_HARNESS_MAX_MEMORY("dataflow_java_harness_max_memory"), JAVA_HARNESS_RESTARTS("dataflow_java_harness_restarts"), TIME_AT_MAX_ACTIVE_THREADS("dataflow_time_at_max_active_threads"), + ACTIVE_THREADS("dataflow_active_threads"), + TOTAL_ALLOCATED_THREADS("dataflow_total_allocated_threads"), WINDMILL_QUOTA_THROTTLING("dataflow_streaming_engine_throttled_msecs"), MEMORY_THRASHING("dataflow_streaming_engine_user_worker_thrashing"); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 7110fee29362..5d4c0288c838 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -246,6 +246,8 @@ public class StreamingDataflowWorker { private final Counter javaHarnessUsedMemory; private final Counter javaHarnessMaxMemory; private final Counter timeAtMaxActiveThreads; + private final Counter activeThreads; + private final Counter totalAllocatedThreads; private final Counter windmillMaxObservedWorkItemCommitBytes; private final Counter memoryThrashing; private final boolean publishCounters; @@ -330,6 +332,11 @@ public class StreamingDataflowWorker { this.timeAtMaxActiveThreads = pendingCumulativeCounters.longSum( StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()); + this.activeThreads = + pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName()); + this.totalAllocatedThreads = + pendingCumulativeCounters.intSum( + StreamingSystemCounterNames.TOTAL_ALLOCATED_THREADS.counterName()); this.windmillMaxObservedWorkItemCommitBytes = pendingCumulativeCounters.intMax( StreamingSystemCounterNames.WINDMILL_MAX_WORK_ITEM_COMMIT_BYTES.counterName()); @@ -1702,6 +1709,10 @@ private void updateVMMetrics() { private void updateThreadMetrics() { timeAtMaxActiveThreads.getAndReset(); timeAtMaxActiveThreads.addValue(workUnitExecutor.allThreadsActiveTime()); + activeThreads.getAndReset(); + activeThreads.addValue(workUnitExecutor.activeCount()); + totalAllocatedThreads.getAndReset(); + totalAllocatedThreads.addValue(chooseMaximumNumberOfThreads()); } @VisibleForTesting diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java index 05b752f91c0c..a160b0e6ad03 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java @@ -119,6 +119,10 @@ public long allThreadsActiveTime() { return totalTimeMaxActiveThreadsUsed; } + public int activeCount() { + return activeCount.intValue(); + } + public String summaryHtml() { monitor.enter(); try { 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 82fc38055a88..24e6e2795c68 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 @@ -291,7 +291,7 @@ static Work createMockWork(long workToken, Consumer processWorkFn) { Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), Instant::now, Collections.emptyList(), - work -> {}); + processWorkFn); } private byte[] intervalWindowBytes(IntervalWindow window) throws Exception { @@ -2793,6 +2793,79 @@ public void testMaxThreadMetric() throws Exception { executor.shutdown(); } + volatile boolean stop = false; + + @Test + public void testActiveThreadMetric() throws Exception { + int maxThreads = 5; + int threadExpirationSec = 60; + // setting up actual implementation of executor instead of mocking to keep track of + // active thread count. + BoundedQueueExecutor executor = + new BoundedQueueExecutor( + maxThreads, + threadExpirationSec, + TimeUnit.SECONDS, + maxThreads, + 10000000, + new ThreadFactoryBuilder() + .setNameFormat("DataflowWorkUnits-%d") + .setDaemon(true) + .build()); + + ComputationState computationState = + new ComputationState( + "computation", + defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + executor, + ImmutableMap.of(), + null); + + ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1); + + Consumer sleepProcessWorkFn = + unused -> { + synchronized (this) { + this.notify(); + } + int count = 0; + while (!stop) { + count += 1; + } + }; + + Work m2 = createMockWork(2, sleepProcessWorkFn); + + Work m3 = createMockWork(3, sleepProcessWorkFn); + + Work m4 = createMockWork(4, sleepProcessWorkFn); + assertEquals(0, executor.activeCount()); + + assertTrue(computationState.activateWork(key1Shard1, m2)); + synchronized (this) { + executor.execute(m2, m2.getWorkItem().getSerializedSize()); + this.wait(); + // Seems current executor executes the initial work item twice + this.wait(); + } + assertEquals(2, executor.activeCount()); + + assertTrue(computationState.activateWork(key1Shard1, m3)); + assertTrue(computationState.activateWork(key1Shard1, m4)); + synchronized (this) { + executor.execute(m3, m3.getWorkItem().getSerializedSize()); + this.wait(); + } + assertEquals(3, executor.activeCount()); + synchronized (this) { + executor.execute(m4, m4.getWorkItem().getSerializedSize()); + this.wait(); + } + assertEquals(4, executor.activeCount()); + stop = true; + executor.shutdown(); + } + @Test public void testExceptionInvalidatesCache() throws Exception { // We'll need to force the system to limit bundles to one message at a time. From 5fd870e1023b7ad591b6d8c5a30fc074673753dc Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 2 Oct 2023 10:51:26 -0400 Subject: [PATCH 55/64] Fix dockerhub task --- .github/workflows/build_release_candidate.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 2d826d572d90..ded089667364 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -159,7 +159,7 @@ jobs: ref: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" repository: apache/beam - name: Free Disk Space (Ubuntu) - uses: jlumbroso/free-disk-space@v1 + uses: jlumbroso/free-disk-space@v1.3.0 - name: Install Java 11 uses: actions/setup-java@v3 with: From 3bbfa7ec29adfb7b09d0199c690157ca630891a2 Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Mon, 2 Oct 2023 18:56:57 +0200 Subject: [PATCH 56/64] runner image to 2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c (#28769) --- .../arc/environments/beam.env | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env index 62a15edf5d6c..9de66b628c89 100644 --- a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env +++ b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env @@ -31,7 +31,7 @@ deploy_webhook = "true" runner_group = "beam" main_runner = { name = "main-runner" - runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:60d397ecfbd2b10a1929615c70d500eb71a2c053" + runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" machine_type = "e2-standard-16" min_node_count = "1" max_node_count = "24" @@ -47,7 +47,7 @@ main_runner = { additional_runner_pools = [{ name = "small-runner" machine_type = "e2-standard-2" - runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:60d397ecfbd2b10a1929615c70d500eb71a2c053" + runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" min_node_count = "1" max_node_count = "10" min_replicas = "1" @@ -64,7 +64,7 @@ additional_runner_pools = [{ { name = "highmem-runner" machine_type = "c3-highmem-8" - runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:60d397ecfbd2b10a1929615c70d500eb71a2c053" + runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" min_node_count = "1" max_node_count = "10" min_replicas = "1" From 4a9608dfc82168591eebe9e3f98ded235ad638f4 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Tue, 3 Oct 2023 02:16:12 +0600 Subject: [PATCH 57/64] Github Workflow Replacement for Jenkins Jobs, beam_PerformanceTests_xlang_KafkaIO_Python (#28730) * beam_PerformanceTests_xlang_KafkaIO_Python * comment description fix * description fix * disable Jenkins job perf test xlang_KafkaIO_Python --- ..._PerformanceTests_xlang_KafkaIO_Python.yml | 121 +++++++++++++++++ .../xlang_KafkaIO_Python.txt | 30 +++++ ...rformanceTests_xlang_KafkaIO_Python.groovy | 124 ------------------ .../04-outside-services/outside-0.yml | 1 - .../04-outside-services/outside-1.yml | 3 +- .../04-outside-services/outside-2.yml | 3 +- 6 files changed, 153 insertions(+), 129 deletions(-) create mode 100644 .github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml create mode 100644 .github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt delete mode 100644 .test-infra/jenkins/job_PerformanceTests_xlang_KafkaIO_Python.groovy diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml new file mode 100644 index 000000000000..a29454ced4bf --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -0,0 +1,121 @@ +# 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: PerformanceTests xlang KafkaIO Python + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 5 * * *' + 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.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PerformanceTests_xlang_KafkaIO_Python: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java CompressedTextIO Performance Test' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PerformanceTests_xlang_KafkaIO_Python"] + job_phrase: ["Run Java CompressedTextIO Performance Test"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores-for-tests + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Kafka + id: install_kafka + run: | + cd ${{ github.workspace }}/.test-infra/kubernetes/kafka-cluster/ + kubectl apply -R -f . + - name: Get Kafka IP + id: kafka_ip + run: | + kubectl wait svc/outside-0 --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + kubectl wait svc/outside-1 --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + kubectl wait svc/outside-2 --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=120s + KAFKA_BROKER_0_IP=$(kubectl get svc outside-0 -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + KAFKA_BROKER_1_IP=$(kubectl get svc outside-1 -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + KAFKA_BROKER_2_IP=$(kubectl get svc outside-2 -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo KAFKA_BROKER_0=$KAFKA_BROKER_0_IP >> $GITHUB_OUTPUT + echo KAFKA_BROKER_1=$KAFKA_BROKER_1_IP >> $GITHUB_OUTPUT + echo KAFKA_BROKER_2=$KAFKA_BROKER_2_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt + arguments: | + --filename_prefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + --bootstrap_servers=${{ steps.kafka_ip.outputs.KAFKA_BROKER_0 }}:32400,${{ steps.kafka_ip.outputs.KAFKA_BROKER_1 }}:32400,${{ steps.kafka_ip.outputs.KAFKA_BROKER_2 }}:32400 + - name: run shadowJar + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:expansion-service:shadowJar + # The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: run integrationTest + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -Prunner=DataflowRunner \ + -PloadTest.mainClass=apache_beam.io.external.xlang_kafkaio_perf_test \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_PerformanceTests_xlang_KafkaIO_Python_test_arguments_1 }}' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt b/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt new file mode 100644 index 000000000000..81702220d3b1 --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--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 +--publish_to_big_query=true +--metrics_dataset=beam_performance +--metrics_table=python_kafkaio_results +--influx_measurement=python_kafkaio_results +--test_class=KafkaIOPerfTest +--input_options=''{\\"num_records\\":100000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--kafka_topic=beam +--read_timeout=1500 +--num_workers=5 +--autoscaling_algorithm=NONE \ No newline at end of file diff --git a/.test-infra/jenkins/job_PerformanceTests_xlang_KafkaIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_xlang_KafkaIO_Python.groovy deleted file mode 100644 index bbab65ed8e4a..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_xlang_KafkaIO_Python.groovy +++ /dev/null @@ -1,124 +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. - */ - -import CommonJobProperties as common -import Kubernetes -import LoadTestsBuilder as loadTestsBuilder -import InfluxDBCredentialsHelper - -def jobs = [ - [ - name : 'beam_PerformanceTests_xlang_KafkaIO_Python', - description : 'Runs performance tests for xlang Python KafkaIO', - test : 'apache_beam.io.external.xlang_kafkaio_perf_test', - githubTitle : 'Python xlang KafkaIO Performance Test', - githubTriggerPhrase: 'Run Python xlang KafkaIO Performance Test', - pipelineOptions : [ - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'python_kafkaio_results', - influx_measurement : 'python_kafkaio_results', - test_class : 'KafkaIOPerfTest', - input_options : """'{ - "num_records": 100000000, - "key_size": 10, - "value_size": 90, - "algorithm": "lcg" - }'""".trim().replaceAll("\\s", ""), - kafka_topic : 'beam', - read_timeout : '1500', - num_workers : '5', - autoscaling_algorithm: 'NONE' - ] - ] -] - -jobs.findAll { - it.name in [ - // all tests that enabled - 'beam_PerformanceTests_xlang_KafkaIO_Python', - ] -}.forEach { testJob -> createKafkaIOTestJob(testJob) } - -private void createKafkaIOTestJob(testJob) { - job(testJob.name) { - description(testJob.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, testJob.githubTitle, testJob.githubTriggerPhrase) - common.setAutoJob(delegate, 'H H * * *') - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Setup kafka k8s pods - String namespace = common.getKubernetesNamespace(testJob.name) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - String kafkaDir = common.makePathAbsolute("src/.test-infra/kubernetes/kafka-cluster") - String kafkaTopicJob = "job.batch/kafka-config-eff079ec" - - /** - * Specifies steps to avoid port collisions when the Kafka outside services (1,2,3) are created. - Function k8s.availablePort finds unused ports in the Kubernetes cluster in a range from 32400 - to 32767 by querying used ports, those ports are stored in env vars like KAFKA_SERVICE_PORT_${service}, - which are used to replace default ports for outside-${service}.yml files, before the apply command. - */ - steps { - String[] configuredPorts = ["32400", "32401", "32402"] - String HIGH_RANGE_PORT = "32767" - (0..2).each { service -> - k8s.availablePort(service == 0 ? configuredPorts[service] : "\$KAFKA_SERVICE_PORT_${service-1}", - HIGH_RANGE_PORT, "KAFKA_SERVICE_PORT_$service") - shell("sed -i -e s/${configuredPorts[service]}/\$KAFKA_SERVICE_PORT_$service/ \ - ${kafkaDir}/04-outside-services/outside-${service}.yml") - } - gradle { - rootBuildScriptDir(common.checkoutDir) - tasks(':sdks:java:io:expansion-service:shadowJar') - } - } - k8s.apply(kafkaDir) - (0..2).each { k8s.loadBalancerIP("outside-$it", "KAFKA_BROKER_$it") } - k8s.waitForJob(kafkaTopicJob,"40m") - - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - bootstrap_servers: "\$KAFKA_BROKER_0:\$KAFKA_SERVICE_PORT_0,\$KAFKA_BROKER_1:\$KAFKA_SERVICE_PORT_1," + - "\$KAFKA_BROKER_2:\$KAFKA_SERVICE_PORT_2", //KAFKA_BROKER_ represents IP and KAFKA_SERVICE_ port of outside services - ] - testJob.pipelineOptions.putAll(additionalPipelineArgs) - - def dataflowSpecificOptions = [ - runner : 'DataflowRunner', - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/', - filename_prefix : "gs://temp-storage-for-perf-tests/${testJob.name}/\${BUILD_ID}/", - sdk_harness_container_image_overrides: '.*java.*,gcr.io/apache-beam-testing/beam-sdk/beam_java8_sdk:latest' - ] - - Map allPipelineOptions = dataflowSpecificOptions << testJob.pipelineOptions - - loadTestsBuilder.loadTest( - delegate, - testJob.name, - CommonTestProperties.Runner.DATAFLOW, - CommonTestProperties.SDK.PYTHON, - allPipelineOptions, - testJob.test) - } -} diff --git a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-0.yml b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-0.yml index e7513ec1b89a..b5dd678c2a23 100644 --- a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-0.yml +++ b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-0.yml @@ -25,5 +25,4 @@ spec: - protocol: TCP targetPort: 9094 port: 32400 - nodePort: 32400 type: LoadBalancer diff --git a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-1.yml b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-1.yml index 50e5fb0650bd..af079b7fdf75 100644 --- a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-1.yml +++ b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-1.yml @@ -24,6 +24,5 @@ spec: ports: - protocol: TCP targetPort: 9094 - port: 32401 - nodePort: 32401 + port: 32400 type: LoadBalancer diff --git a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-2.yml b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-2.yml index 87c324b8eea9..e5c3090673f9 100644 --- a/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-2.yml +++ b/.test-infra/kubernetes/kafka-cluster/04-outside-services/outside-2.yml @@ -24,6 +24,5 @@ spec: ports: - protocol: TCP targetPort: 9094 - port: 32402 - nodePort: 32402 + port: 32400 type: LoadBalancer From 11e2bae4cbee4cc4f9d200a71511d921e8591dcd Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 2 Oct 2023 13:38:48 -0700 Subject: [PATCH 58/64] Fix input detection. (#28775) --- sdks/python/apache_beam/yaml/yaml_provider.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 3a88f6074b37..630e63c31d8a 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -169,8 +169,8 @@ def config_schema(self, type): self.schema_transforms()[self._urns[type]].configuration_schema) def requires_inputs(self, typ, args): - if self._urns[type] in self.schema_transforms(): - return bool(self.schema_transforms()[self._urns[type]].inputs) + if self._urns[typ] in self.schema_transforms(): + return bool(self.schema_transforms()[self._urns[typ]].inputs) else: return super().requires_inputs(typ, args) From a7bf59436a664dca932c5001b750514dab2fa296 Mon Sep 17 00:00:00 2001 From: aku019 <39730533+aku019@users.noreply.github.com> Date: Tue, 3 Oct 2023 03:26:34 +0530 Subject: [PATCH 59/64] Removing Depreciation warning when using pipeline.options (#28774) Co-authored-by: asishupadhyay24@gmail.com --- sdks/python/apache_beam/pipeline.py | 4 ---- .../apache_beam/runners/interactive/interactive_beam.py | 6 ------ .../runners/interactive/interactive_environment.py | 6 ------ .../apache_beam/runners/interactive/recording_manager.py | 5 ----- 4 files changed, 21 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 18b422ed27d4..042b483d50f1 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -235,10 +235,6 @@ def __init__(self, runner=None, options=None, argv=None): @property # type: ignore[misc] # decorated property not supported - @deprecated( - since='First stable release', - extra_message='References to .options' - ' will not be supported') def options(self): # type: () -> PipelineOptions return self._options diff --git a/sdks/python/apache_beam/runners/interactive/interactive_beam.py b/sdks/python/apache_beam/runners/interactive/interactive_beam.py index 878552ef345f..207b0f4e6451 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_beam.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_beam.py @@ -35,7 +35,6 @@ # pytype: skip-file import logging -import warnings from datetime import timedelta from typing import Dict from typing import List @@ -496,11 +495,6 @@ def cleanup( dcm = self.pipelines.pop(p, None) if dcm: dcm.pipelines.remove(p) - warnings.filterwarnings( - 'ignore', - 'options is deprecated since First stable release. References to ' - '.options will not be supported', - category=DeprecationWarning) p_flink_options = p.options.view_as(FlinkRunnerOptions) p_flink_options.flink_master = '[auto]' p_flink_options.flink_version = None diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment.py b/sdks/python/apache_beam/runners/interactive/interactive_environment.py index e508ce351ea3..0e3d0060b1a4 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py @@ -29,7 +29,6 @@ import logging import os import tempfile -import warnings from collections.abc import Iterable from pathlib import PurePath @@ -374,11 +373,6 @@ def get_cache_manager(self, pipeline, create_if_absent=False): given pipeline. If the pipeline is absent from the environment while create_if_absent is True, creates and returns a new file based cache manager for the pipeline.""" - warnings.filterwarnings( - 'ignore', - 'options is deprecated since First stable release. References to ' - '.options will not be supported', - category=DeprecationWarning) cache_manager = self._cache_managers.get(str(id(pipeline)), None) pipeline_runner = detect_pipeline_runner(pipeline) diff --git a/sdks/python/apache_beam/runners/interactive/recording_manager.py b/sdks/python/apache_beam/runners/interactive/recording_manager.py index d123736233fe..bee215717b4d 100644 --- a/sdks/python/apache_beam/runners/interactive/recording_manager.py +++ b/sdks/python/apache_beam/runners/interactive/recording_manager.py @@ -448,11 +448,6 @@ def record(self, pcolls, max_n, max_duration): # incomplete. self._clear() - warnings.filterwarnings( - 'ignore', - 'options is deprecated since First stable release. References to ' - '.options will not be supported', - category=DeprecationWarning) cache_path = ie.current_env().options.cache_root is_remote_run = cache_path and ie.current_env( ).options.cache_root.startswith('gs://') From bd1c95cb0de4dd5b5d5fe34ad9059dc463460841 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Mon, 2 Oct 2023 16:49:36 -0700 Subject: [PATCH 60/64] Better flag documentation. (#28753) --- .../runners/dataflow/dataflow_runner.py | 20 +++++++------------ 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 950bff768ef4..7ad6ab04be68 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -97,10 +97,6 @@ def __init__(self, cache=None): def is_fnapi_compatible(self): return False - def apply(self, transform, input, options): - _check_and_add_missing_options(options) - return super().apply(transform, input, options) - @staticmethod def poll_for_job_completion( runner, result, duration, state_update_callback=None): @@ -496,10 +492,6 @@ def _get_coder(typehint, window_coder): coders.registry.get_coder(typehint), window_coder=window_coder) return coders.registry.get_coder(typehint) - # TODO(srohde): Remove this after internal usages have been removed. - def apply_GroupByKey(self, transform, pcoll, options): - return transform.expand(pcoll) - def _verify_gbk_coders(self, transform, pcoll): # Infer coder of parent. # @@ -589,12 +581,14 @@ def _check_and_add_missing_options(options): sdk_location = options.view_as(SetupOptions).sdk_location if 'dev' in beam.version.__version__ and sdk_location == 'default': raise ValueError( - "When launching Dataflow Jobs with an unreleased SDK, " + "You are submitting a pipeline with Apache Beam Python SDK " + f"{beam.version.__version__}. " + "When launching Dataflow jobs with an unreleased (dev) SDK, " "please provide an SDK distribution in the --sdk_location option " - "to use consistent SDK version at " - "pipeline submission and runtime. To ignore this error and use the " - "SDK installed in Dataflow dev containers, use " - "--sdk_location=container.") + "to use a consistent SDK version at " + "pipeline submission and runtime. To ignore this error and use " + "an SDK preinstalled in the default Dataflow dev runtime environment " + "or in a custom container image, use --sdk_location=container.") # Streaming only supports using runner v2 (aka unified worker). # Runner v2 only supports using streaming engine (aka windmill service) From 71c8459633ec86e576eca080a26be9f42474ecb2 Mon Sep 17 00:00:00 2001 From: pablo rodriguez defino Date: Mon, 2 Oct 2023 17:07:58 -0700 Subject: [PATCH 61/64] [Blog Post] Apache Beam for a content discovery platform (#28734) Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> Co-authored-by: nams113 <39890215+nams113@users.noreply.github.com> --- ...i-content-discovery-platform-genai-beam.md | 338 ++++++++++++++++++ website/www/site/data/authors.yml | 6 + .../blog/dyi-cdp-genai-beam/cdp-arch.png | Bin 0 -> 271543 bytes .../blog/dyi-cdp-genai-beam/cdp-highlevel.png | Bin 0 -> 31242 bytes .../blog/dyi-cdp-genai-beam/pipeline-1.png | Bin 0 -> 146525 bytes .../pipeline-2-extractcontent.png | Bin 0 -> 130427 bytes .../pipeline-3-errorhandling.png | Bin 0 -> 112800 bytes .../pipeline-4-processembeddings1.png | Bin 0 -> 49246 bytes .../pipeline-4-processembeddings2.png | Bin 0 -> 58035 bytes .../pipeline-5-storecontent.png | Bin 0 -> 74751 bytes .../pipeline-6-refresh1.png | Bin 0 -> 74889 bytes .../pipeline-6-refresh2.png | Bin 0 -> 72757 bytes .../pipeline-6-refresh3.png | Bin 0 -> 53972 bytes 13 files changed, 344 insertions(+) create mode 100644 website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-highlevel.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png diff --git a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md new file mode 100644 index 000000000000..8057374591d7 --- /dev/null +++ b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md @@ -0,0 +1,338 @@ +--- +layout: post +title: "DIY GenAI Content Discovery Platform with Apache Beam" +date: 2023-09-27 00:00:01 -0800 +categories: + - blog +authors: + - pabs + - namitasharma +--- + + +# DIY GenAI Content Discovery Platform with Apache Beam + +Your digital assets, such as documents, PDFs, spreadsheets, and presentations, contain a wealth of valuable information, but sometimes it's hard to find what you're looking for. This blog post explains how to build a DIY starter architecture, based on near real-time ingestion processing and large language models (LLMs), to extract meaningful information from your assets. The model makes the information available and discoverable through a simple natural language query. + +Building a near real-time processing pipeline for content ingestion might seem like a complex task, and it can be. To make pipeline building easier, the Apache Beam framework exposes a set of powerful constructs. These constructs remove the following complexities: interacting with multiple types of content sources and destinations, error handling, and modularity. They also maintain resiliency and scalability with minimal effort. You can use an Apache Beam streaming pipeline to complete the following tasks: + +- Connect to the many components of a solution. +- Quickly process content ingestion requests of documents. +- Make the information in the documents available a few seconds after ingestion. + +LLMs are often used to extract content and summarize information stored in many different places. Organizations can use LLMs to quickly find relevant information disseminated in multiple documents written across the years. The information might be in different formats, or the documents might be too long and complex to read and understand quickly. Use LLMs to process this content to make it easier for people to find the information that they need. + +Follow the steps in this guide to create a custom scalable solution for data extraction, content ingestion, and storage. Learn how to kickstart the development of a LLM-based solution using Google Cloud products and generative AI offerings. Google Cloud is designed to be simple to use, scalable, and flexible, so you can use it as a starting point for further expansion or experimentation. + +### High-level Flow + +In this workflow, content uptake and query interactions are completely separated. An external content owner can send documents stored in Google Docs or in a binary text format and receive a tracking ID for the ingestion request. The ingestion process gets the content of the document and creates chunks that are configurable in size. Each document chunk is used to generate embeddings. These embeddings represent the content semantics, in the form of a vector of 768 dimensions. Given the document identifier and the chunk identifier, you can store the embeddings in a Vector database for semantic matching. This process is central to contextualizing user inquiries. + +Content Discovery Platform Overview + +The query resolution process doesn't depend directly on information ingestion. The user receives relevant answers based on the content ingested until the moment of the query request. Even if the platform doesn't have any relevant content stored, the platform returns an answer stating that it doesn't have relevant content. Therefore, the query resolution process first generates embeddings from the query content and from the previously existing context, like previous exchanges with the platform, then matches these embeddings with the existing embedding vectors stored from the content. When the platform has positive matches, it retrieves the plain-text content represented by the content embeddings. Finally, by using the textual representation of the query and the textual representation of the matched content, the platform formulates a request to the LLM to provide a final answer to the original user inquiry. + +## Components of the solution + +Use the low-ops capabilities of the Google Cloud services to create a set of highly scalable features. You can separate the solution into two main components: the service layer and the content ingestion pipeline. The service layer acts as the entry point for document ingestion and user queries. It’s a simple set of REST resources exposed through Cloud Run and implemented by using [Quarkus](https://quarkus.io/) and the client libraries to access other services (Vertex AI models, Cloud Bigtable and Pub/Sub). The content ingestion pipeline includes the following components: + +* A streaming pipeline that captures user content from wherever it resides. +* A process that extracts meaning from this content as a set of multi-dimensional vectors (text embeddings). +* A storage system that simplifies context matching between knowledge content and user inquiries (a Vector Database). +* Another storage system that maps knowledge representation with the actual content, forming the aggregated context of the inquiry. +* A model capable of understanding the aggregated context and, through prompt engineering, delivering meaningful answers. +* HTTP and gRPC-based services. + +Together, these components provide a comprehensive and simple implementation for a content discovery platform. + +## Workflow Architecture + +This section explains how the different components interact. + +### Dependencies of the components + +The following diagram shows all of the components that the platform integrates with. It also shows all of the dependencies that exist between the components of the solution and the Google Cloud services. + +Content Discovery Platform Interactions + +As seen in the diagram, the context-extraction component is the central aspect in charge of retrieving the document’s content, also their semantic meaning from the embedding’s model and storing the relevant data (chunks text content, chunks embeddings, JSON-L content) in the persistent storage systems for later use. PubSub resources are the glue between the streaming pipeline and the asynchronous processing, capturing the user ingestion requests, retries from potential errors from the ingestion pipeline (like the cases on where documents have been sent for ingestion but the permission has not been granted yet, triggering a retry after some minutes) and content refresh events (periodically the pipeline will scan the ingested documents, review the latest editions and define if a content refresh should be triggered). + +The context-extraction component retrieves the content of the documents, diving it in chunks. It also computes embeddings, using the LLM interaction, from the extracted content. Then it stores the relevant data (chunks text content, chunks embeddings, JSON-L content) in the persistent storage systems for later use. Pub/Sub resources connect the streaming pipeline and the asynchronous processing, capturing the following actions: +- user ingestion requests +- retries from errors from the ingestion pipeline, such as when documents are sent for ingestion but access permissions are missing +- content refresh events (periodically the pipeline scans the ingested documents, reviews the latest editions, and decides whether to trigger a content refresh) + +Also, CloudRun plays an important role exposing the services, interacting with many Google Cloud services to resolve the user query or ingestion requests. For example, while resolving a query request the service will: +- Request the computation of embeddings from the user’s query by interacting with the embeddings model +- Find near neighbor matches from the Vertex AI Vector Search (formerly Matching Engine) using the query embeddings representation +- Retrieve the text content from BigTable for those matched vectors, using their identifier, in order contextualize a LLM prompt +- And finally create a request to the VertexAI Chat-Bison model, generating the response the system will delivery to the user’s query. + +### Google Cloud products + +This section describes the Google Cloud products and services used in the solution and what purpose they serve. + +**Cloud Build:** All container images, including services and pipelines, are built directly from source code by using Cloud Build. Using Cloud Build simplifies code distribution during the deployment of the solution. + +**CloudRun:** The solution's service entry points are deployed and automatically scaled by CloudRun. + +**Pub/Sub:** A Pub/Sub topic and subscription queue all of the ingestion requests for Google Drive or self-contained content and deliver the requests to the pipeline. + +**Dataflow:** A multi-language, streaming Apache Beam pipeline processes the ingestion requests. These requests are sent to the pipeline from the Pub/Sub subscription. The pipeline extracts content from Google Docs, Google Drive URLs, and self-contained binary encoded text content. It then produces content chunks. These chunks are sent to one of the Vertex AI foundational models for the embedding representation. The embeddings and chunks from the documents are sent to Vertex AI Vector Search and to Cloud Bigtable for indexing and rapid access. Finally, the ingested documentation is stored in Google Cloud Storage in JSON-L format, which can be used to fine-tune the Vertex AI models. By using Dataflow to run the Apache Beam streaming pipeline, you minimize the ops needed to scale resources. If you have a burst on ingestion requests, Dataflow can keep the latency less than a minute. + +**Vertex AI - Vector Search:** [Vector Search](https://cloud.google.com/vertex-ai/docs/matching-engine/overview) is a high-performance, low-latency vector database. These vector databases are often called vector similarity search or approximate nearest neighbor (ANN) services. We use a Vector Search Index to store all the ingested documents embeddings as a meaning representation. These embeddings are indexed by chunk and document id. Later on, these identifiers can be used to contextualize the user queries and enrich the requests made to a LLM by providing knowledge extracted directly from the document’s content mappings stored on BigTable (using the same chunk-document identifiers). + +**Cloud BigTable:** This storage system provides a low latency search by identifier at a predictable scale. Is a perfect fit, given the low latency of the requests resolution, for online exchanges between user queries and the platform component interactions. It used to store the content extracted from the documents since it's indexed by chunk and document identifier. Every time a user makes a request to the query service, and after the query text embeddings are resolved and matched with the existing context, the document and chunk ids are used to retrieve the document’s content that will be used as context to request an answer to the LLM in use. Also, BigTable is used to keep track of the conversational exchanges between users and the platform, furthermore enriching the context included on the requests sent to the LLMs (embeddings, summarization, chat Q&A). + +**Vertex AI - Text Embedding Model:** [Text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text-embeddings) are a condensed vector (numeric) representation of a piece of text. If two pieces of text are semantically similar, their corresponding embeddings will be located close together in the embedding vector space. For more details please see [get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). These embeddings are directly used by the ingestion pipeline when processing the document’s content and the query service as an input to match the users query semantic with existing content indexed in Vector Search. + +**Vertex AI - Text Summarization Model:** [Text-bison](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text) is the name of the PaLM 2 LLM that understands, summarizes and generates text. The types of content that text-bison can create include document summaries, answers to questions, and labels that classify the provided input content. We used this LLM to summarize the previously maintained conversation with the goal of enriching the user’s queries and better embedding matching. In summary, the user does not have to include all the context of his question, we extract and summarize it from the conversation history. + +**Vertex AI - Text Chat Model:** [Chat-bison](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text-chat) is the PaLM 2 LLM that excels at language understanding, language generation, and conversations. This chat model is fine-tuned to conduct natural multi-turn conversations, and is ideal for text tasks about code that require back-and-forth interactions. We use this LLM to provide answers to the queries made by users of the solution, including the conversation history between both parties and enriching the model’s context with the content stored in the solution. + +### Extraction Pipeline + +The content extraction pipeline is the platform's centerpiece. It takes care of handling content ingestion requests, extracting documents content and computing embeddings from that content, to finally store the data in specialized storage systems that will be used in the query service components for rapid access. + +#### High Level View + +As previously mentioned the pipeline is implemented using Apache Beam framework and runs in streaming fashion on GCP's [Dataflow](https://cloud.google.com/dataflow) service. + +By using Apache Beam and Dataflow we can ensure minimal latency (sub minute processing times), low ops (no need to manually scale up or down the pipeline when traffic spikes occur with time, worker recycle, updates, etc.) and with high level of observability (clear and abundant performance metrics are available). + +Apache Beam Pipeline + +On a high level, the pipeline separates the extraction, computing, error handling and storage responsibilities on different components or PTransforms. As seen in the diagram, the messages are read from a PubSub subscription and immediately afterwards are included in the window definition before the content extraction. + +Each of those PTransforms can be expanded to reveal more details regarding the underlying stages for the implementation. We will dive into each in the following sections. + +The pipeline was implemented using a multi-language approach, with the main components written in the Java language (JDK version 17) and those related with the embeddings computations implemented in Python (version 3.11) since the Vertex AI API clients are available for this language. + +#### Content Extraction + +The content extraction component is in charge of reviewing the ingestion request payload and deciding (given the event properties) if it will need to retrieve the content from the event itself (self-contained content, text based document binary encoded) or retrieve it from Google Drive. + +Pipeline's Content Extraction + +In case of a self-contained document, the pipeline will extract the document id and format the document in paragraphs for later embedding processing. + +When in need of retrieval from Google Drive, the pipeline will inspect if the provided URL in the event refers to a Google Drive folder or a single file format (supported formats are Documents, Spreadsheets and Presentations). In the case of a folder, the pipeline will crawl the folder’s content recursively extracting all the files for the supported formats, in case of a single document will just return that one. + +Finally, with all the file references retrieved from the ingestion request, textual content is extracted from the files (no image support implemented for this PoC). That content will also be passed to the embedding processing stages including the document’s identifier and the content as paragraphs. + +#### Error Handling + +On every stage of the content extraction process multiple errors can be encountered, malformed ingestion requests, non-conformant URLs, lack of permissions for Drive resources, lack of permissions for File data retrieval. + +In all those cases a dedicated component will capture those potential errors and define, given the nature of the error, if the event should be retried or sent to a dead letter GCS bucket for later inspection. + +Pipeline's Error Handling + +The final errors, or those which won’t be retried, are those errors related with bad request formats (the event itself or the properties content, like malformed or wrong URLs, etc.). + +The retryable errors are those related with content access and lack of permissions. A request may have been resolved faster than the manual process of providing the right permissions to the Service Account that runs the pipeline to access the resources included in the ingestion request (Google Drive folders or files). In case of detecting a retryable error, the pipeline will hold the retry for 10 minutes before re-sending the message to the upstream PubSub topic; each error is retried at most 5 times before being sent to the dead letter GCS bucket. + +In all cases of events ending on the dead letter destination, the inspection and re-processing must be done in a manual process. + +#### Process Embeddings + +Once the content has been extracted from the request, or captured from Google Drive files, the pipeline will trigger the embeddings computation process. As previously mentioned the interactions with the Vertex AI Foundational Models API is implemented in Python language. For this reason we need to format the extracted content in Java types that have a direct translation to those existing in the Python world. Those are key-values (in Python those are 2-element tuples), Strings (available in both languages), and iterables (also available in both languages). We could have implemented coders in both languages to support custom transport types, but we opted out of that in favor of clarity and simplicity. + +Before computing the content’s embeddings we decided to introduce a Reshuffle step, making the output consistent to downstream stages, with the idea of avoiding the content extraction step being repeated in case of errors. This should avoid putting pressure on existing access quotas on Google Drive related APIs. + +The pipeline will then chunk the content in configurable sizes and also configurable overlapping, good parameters are hard to get for generic effective data extraction, so we opted to use smaller chunks with small overlapping factor as the default settings to favor diversity on the document results (at least that’s what we see from the empirical results obtained). + +

    + Embeddings Processing + Embeddings Processing +

    + +Once the embeddings vectors are retrieved from the embeddings Vertex AI LLM, we will consolidate them again avoiding repetition of this step in case of downstream errors. + +Worth to notice that this pipeline is interacting directly with Vertex AI models using the client SDKs, Apache Beam already provides supports for this interactions through the RunInference PTransform (see an example [here](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/inference/vertex_ai_llm_text_classification.py)). + +#### Content Storage + +Once the embeddings are computed for the content chunks extracted from the ingested documents, we need to store the vectors in a searchable storage and also the textual content that correlates with those embeddings. We will be using the embeddings vectors as a semantic match later from the query service, and the textual content that corresponds to those embeddings for LLM context as a way to improve and guide the response expectations. + +Content Storage + +With that in mind is that in mind we split the consolidated embeddings into 3 paths, one that stores the vectors into Vertex AI Vector Search (using simple REST calls), another storing the textual content into BigTable (for low latency retrieval after semantic matching) and the final one as a potential clean up of content refresh or re ingestion (more on that later). The three paths are using the ingested document identifier as the correlating data on the actions, this key is formed by the document name (in case of available), the document identifier and the chunk sequence number. The reason for using identifiers for the chunk comes behind the idea of subsequent updates. An increase in the content will generate a larger number of chunks, and upserting all the chunks will enable always fresh data; on the contrary, a decrease in content will generate a smaller chunk count for the document’s content, this number difference can be used to delete the remaining orphan indexed chunks (from content no longer existing in the latest version of the document). + +#### Content Refresh + +The last pipeline component is the simplest, at least conceptually. After the documents from Google Drive gets ingested, an external user can produce updates in them, causing the indexed content to become out of date. We implemented a simple periodic process, inside the same streaming pipeline, that will take care of the review of already ingested documents and see if there are content updates needed. We use a GenerateSequence transform to produce a periodic impulse (every 6 hours by default), that will trigger a scan on BigTable retrieving all the ingested document identifiers. Given those identifiers we can then query Google Drive for the latest update timestamp of each document and use that marker to decide if an update is needed. + +In case of needing to update the document’s content, we can simply send an ingestion request to the upstream PubSub topic and let the pipeline run its course for this new event. Since we are taking care of upserting embeddings and cleaning up those that no longer exist, we should be capable of taking care of the majority of the additions (as long those are text updates, image based content is not being processed as of now). + +

    + Content Refresh + Content Refresh + Content Refresh +

    + +This task could be performed as a separate job, possibly one that is periodically scheduled in batch form. This would result in lower costs, a separate error domain, and more predictable auto scaling behavior. However, for the purposes of this demonstration, it is simpler to have a single job. + +Next, we will be focusing on how the solution interacts with external clients for ingestion and content discovery use cases. + +## Interaction Design + +The solution aims to make the interactions for ingesting and querying the platform as simple as possible. Also, since the ingestion part may imply interacting with several services and imply retries or content refresh, we decided to make both separated and asynchronous, freeing the external users of blocking themselves while waiting for requests resolutions. + +### Example Interactions + +Once the platform is deployed in a GCP project, a simple way to interact with the services is through the use of a web client, curl is a good example. Also, since the endpoints are authenticated, a client needs to include its credentials in the request header to have its access granted. + +Here is an example of an interaction for content ingestion: + +``` +$ > curl -X POST -H "Content-Type: application/json" -H "Authorization: Bearer $(gcloud auth print-identity-token)" https:///ingest/content/gdrive -d $'{"url":"https://drive.google.com/drive/folders/somefolderid"}' | jq . + +# response from service +{ + "status": "Ingestion trace id: " +} +``` + +In this case, after the ingestion request has been sent to the PubSub topic for processing, the service will return the tracking identifier, which maps with the PubSub message identifier. Note the provided URL can be one of a Google Doc or a Google Drive folder, in the later case the ingestion process will crawl the folder’s content recursively to retrieve all the contained documents and their contents. + +Next, an example of a content query interaction, very similar to the previous one: + +``` +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"summarize the benefits of using VertexAI foundational models for Generative AI applications", "sessionId": ""}' \ + | jq . + +# response from service +{ + "content": "VertexAI Foundation Models are a set of pre-trained models that can be used to accelerate the development of machine learning applications. They are available for a variety of tasks, including natural language processing, computer vision, and recommendation systems.\n\nVertexAI Foundation Models can be used to improve the performance of Generative AI applications by providing a starting point for model development. They can also be used to reduce the amount of time and effort required to train a model.\n\nIn addition, VertexAI Foundation Models can be used to improve the accuracy and robustness of Generative AI applications. This is because they are trained on large datasets and are subject to rigorous quality control.\n\nOverall, VertexAI Foundation Models can be a valuable resource for developers who are building Generative AI applications. They can help to accelerate the development process, reduce the cost of development, and improve the performance and accuracy of applications.", + "previousConversationSummary": "", + "sourceLinks": [ + { + "link": "", + "distance": 0.7233397960662842 + } + ], + "citationMetadata": [ + { + "citations": [] + } + ], + "safetyAttributes": [ + { + "categories": [], + "scores": [], + "blocked": false + } + ] +} +``` + +The platform will answer the request with a textual response from the LLM and include as well information about the categorization, citation metadata and source links (if available) of the content used to generate the response (this are for example, Google Docs links of the documents previously ingested by the platform). + +When interacting with the services, a good query will generally return good results, the clearer the query the easier it will be to contextualize its meaning and more accurate information will be sent to the LLMs to retrieve answers. But having to include all the details of the query context in a phrase on every exchange with the service can be very cumbersome and difficult. For that case the platform can use a provided session identifier that will be used to store all the previous exchanges between a user and the platform. This should help the implementation to better contextualize the initial query embeddings matching and even provide more concise contextual information in the model requests. Here is an example of a contextual exchange: + +``` +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"summarize the benefits of using VertexAI foundational models for Generative AI applications?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "VertexAI Foundational Models are a suite of pre-trained models that can be used to accelerate the development of Generative AI applications. These models are available in a variety of languages and domains, and they can be used to generate text, images, audio, and other types of content.\n\nUsing VertexAI Foundational Models can help you to:\n\n* Reduce the time and effort required to develop Generative AI applications\n* Improve the accuracy and quality of your models\n* Access the latest research and development in Generative AI\n\nVertexAI Foundational Models are a powerful tool for developers who want to create innovative and engaging Generative AI applications.", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"describe the available LLM models?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "The VertexAI Foundational Models suite includes a variety of LLM models, including:\n\n* Text-to-text LLMs: These models can generate text based on a given prompt. They can be used for tasks such as summarization, translation, and question answering.\n* Image-to-text LLMs: These models can generate text based on an image. They can be used for tasks such as image captioning and description generation.\n* Audio-to-text LLMs: These models can generate text based on an audio clip. They can be used for tasks such as speech recognition and transcription.\n\nThese models are available in a variety of languages, including English, Spanish, French, German, and Japanese. They can be used to create a wide range of Generative AI applications, such as chatbots, customer service applications, and creative writing tools.", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"do rate limit apply for those LLMs?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "Yes, there are rate limits for the VertexAI Foundational Models. The rate limits are based on the number of requests per second and the total number of requests per day. For more information, please see the [VertexAI Foundational Models documentation](https://cloud.google.com/vertex-ai/docs/foundational-models#rate-limits).", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"care to share the price?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "The VertexAI Foundational Models are priced based on the number of requests per second and the total number of requests per day. For more information, please see the [VertexAI Foundational Models pricing page](https://cloud.google.com/vertex-ai/pricing#foundational-models).", + … +} +``` + +**Usage Tip:** in case of abruptly changing topics, sometimes is better to use a new session identifier. + +### Deployment + +As part of the platform solution, there are a set of scripts that help with the deployment of all the different components. By running the `start.sh` and setting the right parameters (GCP project, terraform state bucket and name for the platform instance) the script will take care of building the code, deploying the needed containers (service endpoint container and Dataflow python custom container), deploying all the GCP resources using Terraform and finally deploying the pipeline. There is also the possibility of modifying the pipeline’s execution by passing an extra parameter to the startup script, for example: `start.sh "--update"` will update the content extraction pipeline in-place. + +Also, in case of wanting to focus only on the deployment of specific components other scripts have been included to help with those specific tasks (build the solution, deploy the infrastructure, deploy the pipeline, deploy the services, etc.). + +### Solution's Notes + +This solution is designed to serve as an example for learning purposes. Many of the configuration values for the extraction pipeline and security restrictions are provided only as examples. The solution doesn't propagate the existing access control lists (ACLs) of the ingested content. As a result, all users that have access to the service endpoints have access to summarizations of the ingested content from those original documents. + +### Notes about the source code + +The source code for the content discovery platform is available in [Github](https://github.com/prodriguezdefino/content-dicovery-platform-gcp). You can run it in any Google Cloud project. The repository includes the source code for the integration services, the multi-language ingestion pipeline, and the deployment automation through Terraform. If you deploy this example, it might take up to 90 minutes to create and configure all the needed resources. The README file contains additional documentation about the deployment prerequisites and example REST interactions. diff --git a/website/www/site/data/authors.yml b/website/www/site/data/authors.yml index 0458bda2c963..2776132cf586 100644 --- a/website/www/site/data/authors.yml +++ b/website/www/site/data/authors.yml @@ -269,3 +269,9 @@ riteshghorse: yhu: name: Yi Hu email: yhu@apache.org +pabs: + name: Pablo Rodriguez Defino + email: prodriguezdefino@gmail.com +namitasharma: + name: Namita Sharma + email: namitasharma@google.com diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png new file mode 100644 index 0000000000000000000000000000000000000000..ef42340e905f5bcab8f6972e42600f3f48690c40 GIT binary patch literal 271543 zcmeFZcT`kq(>;n2P(c|pQ6&io2$Hi2Y!Fa_WDrquP7O^|QB;x=6(lG*i%4pcf`EX~ zN(GEDacS9pgurCLPCMM zaZQPY5)v*(94;r|MVQVVYiFw z!jZlELV8cCm*q8fBy#gIh=Br{(pXV=Gl=W$cp{@^N8$!FZ=g;ko@n0>{t$x|6P#(-@A<0J}k*6 zx+=wXhjlOUh6g+M*Hhr0)Ry|2I>YOloDaF;uTve7&SrBcRK4YNbE+gyCPKi9!d`gi zS7Q7|X-_9RAFjLeV_N^kvx}tIw}vN`bai#}SbG{7S!^|@#%8eN zWKD<))d6}@%e2ioNvn`L!-MuFJ8u-@XH!TJaq9IHFju$odSX=fHi0xDaUb@R;Z>IC{zr*pWVSH!O!0?u=7hK zDrRlt9UnCr9kTM(_dTiFt)mdcc*d6{YjouTs>%2Cr9Cg|<#&@MsM4Q?1BF>C z7JtZo6xln<1hC$W$ z5B%2@NOC@Qn8S#eQD9zsU_9r}_)3e~e!3OdQizV03fsN&?zfEgkWvO8bKTYVLF%Mu zZ)3I^PAbfk<%ey-Co7@^s_BD%`F(+iJvx7xX1Od`qLj?d)84;-AHIRH%+or_rG0Tx z>qq#GJ2tf3GcX7V3!@q44X53u&-Z4M;KFC>di(aP0|yT-RA(7rv-C<&92D6f@#xX+ zNKthCR91UCV(l6iLaB$WMItp<>=T&S%m@efdPTfJpDFxkNx&w{g>eO%Qubk@@Q-9J zg>Sg@+l%BZ&s_i5HBo6K+?AU*owCCOr22K5la*uSj8E+KqPnG!sGPaJg6qsS3ZV7> z;lc22Woa;G*(p9=URhgPJEpdE^RT3fw?Ew;T_4t4qukeu-MVr$JtDpsW}AF}Ejk~R zW$=La&29|$G{t@VSNLV6?;F^thH+foQ9eZ!D%p1Yuo-^U>oI9XRo?WA9RndTy&`pL z7mSu4T~Et+O64x5%%b;0cQP|RX57Ag z+un=f<>i@~kJ@IpVV_Ete;gE!R6s!-$QXAl4CT~fPvuwg_+De$R$q_Xa?gQZ5Pn|_#1y~cvq+NxNBl@v@hf_**Joi-Qls2*WrtfQ9LL(T3`EeAZza}y_biOJDiaEq zUe+1Tk<+;xJ9cciyTGo}CagC8D67J-mCIt1OaMK#!`g(zLcTWFL|2|7gZuPDrjeH9 zpw42KJSLO&5BjS0;+UV$>Z8O{%gU})R93PlSm{-K{kmLSRWm0PsX!;@bPM^CwC|~R zU&zUM-`uQRxqkgR`-KZFT2>pXfxh8PD--jh0TuTems?aS<|oY3#vncg=S=Qnd_7uV zCT&X?m@5ltDDDC<_Yw zND@Y}N->JT1qBA4OKeX5@#96tcLLU-xxjA7v{(=)+MceJCqkNCXP6E9+q~woRAZTE zRvV!;di;o^NaX5#kfMF>=UofKjaN5F4^6Law8MN60El)&0Ya$i>%YTgahI-_2~BQn z77#4y#H>Fr)2l3vvuDbHfngmdQVyQv8lF#99&N~q{qf_6FaJ_n*pSQOh@_J;Z>@54Q|LNAz6*@n1`97bzx?YB{`?8Nk#+xkOpIjar8PLVro|CM@9~hvKmy?+~=*nmvt&9AyG1ECOO%?s;Jn zx7(F`$@MRXbM(tGV9Nso1Eyb}QBm{SCtWz~Y5S6y0+kPDly;mG{fcfPH1+}w{&RN;ir^{lKc ziOY3yarY^T2SWEt`sC*eX{5gbBMJ%1Zk-Zw?t;K#P}kJVpt7}V4--lzfR^VV8lS9wy< z{G|$&sjKCJX|zH-z);idKxx~%b=Ts>(bO9C{0}w7LbMf{)5cG6MUtmQ?;M?adj=)q zG##U)q;%}~@%W^q^NeT2=0mkc3Bvgwzka>OD5j+)(@EEbzLC$k{N1nnSZS+$VbM!eVdLMOD|oeh}*Ql(f5Z_j2MqO?|=NG-)y2}g1k`fH~}j@ zC99$m43nDom@L%;1YMlyGNPSx#}_Y&_rG}Y!uQtXw^w;=?Cdo{lXTLHlx}M7j$I^Xn3VEH3K@-yllMdM+@gZ(fmY|y_zbjPe^1}t z!x43cdp*S8_0#|kq}qGYiyQjJy||c)BrSuEg?<+uH9J3H#1r4YKlealA$VrvH2>(L zX4N$r84e(qHJVG0h-X$lKitEN%Cy3y>*lCb&5559vW_7d2?D5~j&BKmE)4pZ$F|h1 zY%uLyBfhjOmYShcNZobEERKqDaoS!eV{Ga4P8Zrm=hbNH{3vkG{zLOF-G$jBtX2Li z6gL*<4pd?087_Wq2h`1RG+CgeTO1{HO$kdD%}7Vjad zBiqOfm>&|DkofF8H@KE&_u$EsCzZLSbKqX1M~CNin+fKTw!ZQQd0h@azv-7&i9b}cin^Nx+=95ZS)%CKe$+Tu zlX0WOCGeAJYqmt6^ANtC9Vf}}a3w^lKoXp2Xv3jETVmHjR53Wu=YH;xJXeo=4<9T(s#(U)!I_kxTD>sjE1b_i-@sU*XaLPAHv?`WMu_J`~-s0 z-)1UJAB6_`#w3(x6oW~-KYm9=MYWJpA-Y%8rEO16l*ejIGP{1qDE12m8!eEixV4g; zH9_!ejRyZm3>O?4Vq4_~!VLnB9!=dVA-SR!*4}v~n!X3e1|+R?$jLjk_%TsZ;id`% ztytwWR11VAW>x!8c{<6AN^Hu(1`MfgF+tq<#T|)A#Bs1gQ7(Q<-beqyYRgXsY~IsPr; z&OFOl4pPYqXKd);zyP~JU)lb;K7vUuz(s2m9>OEbh<|6Wn2uHE2?n?MS8*?H8Ois_ zCBN4iTcx0}A8+TeAm-MQVTOJOUI01_WnRtVY;}mG6EJ_B{f1j&< zfnwCvEU#<9r^3h2vI`>CgMzmAKu92w3PQZP8msKF(5hDHsB>je&voW{ab_PuW2gDw z5=VGKkB*Jcix(;O%?ninwj*DkAC>mjoKnbR^=HUzDrk$%ihMCvCK^*TTF|;MI!SG8 znb$@3td0Mx8Sk58Cr>5(}xNPmYjNNQIjL}hA9q4&Cgm@6qu1Hj3&yL6yei*=C2v!L=5ky>DNCKn_2$QBb zWY2xi^k3cLuUZ!4cX$*xTDUh_+@-DEf21p~dQ{S3G!EyE&(L|0BP|=axM2`YKT`HpqiD7YI<(WC~7gM4>KLl zkFuutI>Q6`!1posUv~FjQhd48GNSCBX}8)^Dw#>U?*HNiZzY4c?N_n^U^2z3gQt1! zUGvL9HNj0Mwx-MX@6)mnh`3!_kM7}V!&qOrQaofzHq~I4XeOd6?7libB2uT^jBbwT zFc+9yiVcvMb@4~PCaw{7ECvZ5=k7AHX9&emDdZ8=k_%l|+P80?^pWPHNs)qRe*r@y zxxl6bL_K)RQw1nS-38lz`|{<7aWC%vgpq9Ao4JQNep!CX#t*`-pBd`7j=fP`6mX8| zw<|5m(xf_y;&wyd81aJD<~-rksIi>&tA^T=V3((W`X^mEC7iogegzZ33R7P^$GbZ(rUmdR(fJb)^tJ zwWv``C`6l0bn+vYOKh#r?H5lhWL12{l&(`KtyCd8PT0T-gJQ#YZL3YogMYFv+aKo7 zGpQGd6YWaZ5;SnZMTt9Xci3pLZlc zLql_6JiDg0wiS34euq|L8)n==kNyxic4)~ zKO&ExZAf$AUfyeX7>NC18mZ}9Rxofyp*$8yFc2d3@qwIQ%m>!>F+xR0VQsbwEb??; zIV2f5U=*UcQ!kkyU{qq6SXiisswyhj;M+)Y2PrmWHPpu#rsmw$(_cx@`4szMHymN{ zD%F=jo^dZvVhKW(>XvA4ODG87IA~~41Np#h#3f1&AHBFliECSZ`;=m?V+*%clrwp_ zfP7tur%}BM80xe}72HKLcU%qgvu_0?uyFCQUChkP90q+?K$AFSm*y&>%n@DB=ikZ% zY!w`@5{-fRjV_GuG;{gKlx1Y6!0COT9L%|2tpxX2v%j=(>KH>NYjjPTz-u7~({RYd zo5`jy-j6|JSm+*YmH5(A`JuyHHbS6oiU%n3Ky7pbXmY-5Lc!78V12$WmJ6f%y&rQB z%M~mA`LSepbQI)ioKAdKfp(`yCnVxi6V6`srpCzyQauf2FRO&5PH#%@xR*6t(3wk{|QYyfvg{rhPcK&q;ln3eSE>3od-VF~IxeVDuO75QY4)aBg2w0Ic8*z60 zU_mtU9F31E{mRO2mG&b|@%3W^IzDxVW*^a68K6$g|5Sb>O#xb)-;&MD54~^qc}*ZU zmX(!lgM_}n2UHT6Z1!dRcO)5E`pgLX*=E%)wRCsKvKbqmg`9TaA=mEot}aouS;vc; zu@ROAARU3&_NHzS<4s1e%;VC|Z-~i~uTz$T;205SFx^@+VqY_0yJF9kQTuv&+8VAD ziJa1q^Qll8;^B0b%xRUdt2_h-B|p1id{#P}W{5+WOX0$UFZj@mp!_n(R;=Tpy%Bz*5$ zX8K*1lv#~kO-*BJcfcgcr?Y##cEvo0Si3Mukb#tawYoj)As6Y6$2C+!3DlD%INQP{ zMsCO7kbJU1H2eR2q?*s3`Hyr@vX^Ri*?%uP@;|%`Sv!&jO#s~F46)2N808tc|2 z0Or=_bLrR@tGZJfnQY0H?M%fl<2X4}F(E}*~ z$%>c-gogAmeJ~RRjlfOFhw>M{>>-`68>p4QkF=(SDH_`?O%hgrw(~@9)Yv%>iaL(J zR-l#4*eMjhRdbF4HVsadoRo9}(TWjq>GA%GF+g;|$?8i>#KrMOs0*kTIO+hlg1T}Q z$S*U>s&pY#QN2qG*hPJ$XyReXWhsCs!NoCFK*6Qc&h-;ua71+3fbD5qP2rwwqqjyB z+FW6f0KA9F&L3%;zb6qAzD7nF6)?tkdEE%JcbMqRX>$O^AmXb| z&XRME6P+W72n_0 zh0&)UyGoS-0RXhahYK?2FWaLy3@ znMM$pEzoY=&+L7OE(HOLHLfr@1ky&|i161<2`N$_Nw@+56~yxSXlIm*kw}`28^Eliv~^%Fr)!&Z29d}@)8gZtnI*y$>>1`E8#8o z{jCH$d0|97z1C#Lln(PdprnEH*XlnZnsS)Tig*@< zSO638t_^$vDSSX>YYa*OowI`{AV0x#pWtqT^z}aP8*suyNC1+E3}QY_Fof)o&uWsk zAA^iMTp`k4zRDjn_{QibBzg#meP>`574wl*Y@#64NL-;)Q8tBaFM6t{eE4e-U=g_f z9p>+v5eB-1moolE=`}MI0o=U1KSUW#>LeP`@t)vH!#kMSE>G(l^nj)`U(T@JKo&pr zTZNYBn+9sT1{L50`R16*Gnmb#R3!46=C9}#MZ3WanVFc9#vJe?6L~#K$jVPR78g8_ z%h3tZ>;CXXohG6#Va#QbuZNh|vziLUv#tpN5rKfycHTUWE3>RBDk^b0g*~mHQ$xr` z0uYp^E*XAr>GkqrC+-2Z1XTb81Vtc5L~U)D!tD`@fkNJGc(xz?rG(1=8_!`pbKt-M z-;4};xEmt4mqzM~gNuW1s20y*g}9Qa<6EkP1tZWVPT@sCF_NK=%7kLUEfBw3Qq?l4 z%gbV`d~d^ifM2kmKcA!&ORy9iJq=%DG6As<641<^NbA`5@A>`_N zbk^}N8j@rN6uyiJnm24|6(~VH1*z2D5zoWch;IwGJXW%@FQEL3LCqH_7r!KedYq12n@b*6-f0cx2U5r zWFbgOFn{{eD4oJ$i?Zu?3Qvu*+kQXqkZf)?qx%tA!rx^szR8WV9r6Qs9Fs!1j^X=3Q z&pRTt*Q@czU|pnv*VXtJl&NTix9VnOpJHO90yw{4Tpa5(X)lMcS)Itc5OJIXoPdVz zwm68Xkf9uH_4w?`J_qiP{DFvOfs10_)mOT;;b!1pb5;&2o>l&IbEge>-^?e5>88wu zLy?YA=xx+y43V-Lf|B6wIEBP!D|M{xu#kvIXM2gm@-#c{b90X4P(3=uH{`y3Y`Vh0 znd~ZW2G?5_AmXAxI43v>6gn1bS>Ol5p|n{#RWZzCkm4#aE>rI7{~5~ua23E!4>R{j-$oMBZ z48*cWOAZ@mYXAVt-n=9EHrb>=s@p>qy*tY7Cq}v9nFdQUcTvVH^lQW z4t~4bcC4>d?Qb|HGc#^T%&2PjY%E708oS<=Xy8}b1U%m{OQTwDtq?Kl-kOp?X( zfx0z4N%QlhZdXDP(=qQ*f;w6rVo6+KwMOHFpu`)2NGdnsND~b?%|v_2)xh9Tfx2Ze z9-i9(Jxn7Is5&7K}p}cc8CDA!KdQ>%~SZQ}xiBC}FpR%lHh1Ox@y?3>{)gW$u!uiwb2wwb==V zsiiq6d%$X+e%Jx3P`~6<&hp^wgHp(FSeM;0p(Lk^YC-~X(c@oa0k%BTfucxsk5RhI z7bnHBk5CrTKx}v8DTS<}$1SDnWF_yW1a_=n%t+(qQ@esd%pbxlKu^S6*MnRZoYhX4!i$To7A|3+23(58oo%214~4cmgs%SO%c4-xbBiw5h! zk9_Mh!8C)AYy%;0HvS;<1L7cw3##dt19{0;`4(ISdn>N*8_X{ zYQia@u18(vMCvP!MNgeYNqc_!2)p~>g0}jA9gy6}siIglUFYja3q*;>c=POlqKYuL zfIx01MvUMSsV6%2N!sBR>_L2NCX)lCf;C2oijlmzjOWg+*W(M-rcBG5*Cf|((Fmk1 zjfCo`31W(6dc-0_IMDGD5S!y(45{F!0%5YiW#?BD@V-lV8Bm*<9}MM`KEIQBTRT9T z#ECE{G*5IcWB>^ogZdu;b?tAA=<8~3_zoaqgRF_GZyqb29*_0>jZ-LA*sR%p9-=~%$5Fy2Lcg(z4I^_ zo}LxZR8fIoE?Q%>NMu;Tm9MR-(Lp1NL6!f=3Ndq$T->F^!(n7MD_QfnsC|*?v6h z+i`+PK^A0yhURJ~qa^HBZ-(F?dQed!Wk3+%)`|jPE&Ni=poE># zX#*KkXFg9?V!w?}*+LH?Mm+LqdfPW?hv}Uu4Iek~HbkRAAT%y631DNJgEpRaf}(U& zQ$n&{sk`d#Qt1Cd(l5ZpcW!HoCQw4A;?)I&2BC6}oEzX(>sQ-Ij^p?S(0uByu#$qG zcHf&#+Ci@x5}i5^#ghIxto^MV^$W5n)GNU5q5?an4N)O2HE+s+yl~IyC8EdFk{ynk zUf8yM$|kRCqB%bIW4ZT6z?{9E<$+(NAjN2rfu$A9qk)(>Fw(q>$t*(OY{=R72nd0m8?iYe|$5!c)WtqurljY`RF`vnS&(!NO_C#;Xv4%n)m z9-=d}xG*(64&gJEb_wl_s4N%uo?knB>ehe+(q08j*L0@eC&dc|#NIdS7eQT8zICfNTheUN zydMi)0RzgV5K4bkokVuzLvy~qPr^lw{2>Z&bNW|Iz;S8b zA(iR`q<$Mo{2j{`(W==U+5<(=-0CsVZ&3>lSo57o*S-i{Hpl3s9%Hss*Y{5ye2oRIOiLAt~f?U(>fmwZAX=iZ&^qMMl zQCv9m{vl?UBfVNg+S(M8vBAS32Ms!nVDP6R)r!lrf2TQb zHOs>xYvFnH4*q|}1tAc2scHz;VLYaE0qCOdPQT_spXXt#Ho6I!m)}`DLK&e!1OaSU zY*i;n3L>ugmHu=?f@$4>Vwa75D&C8-Dx~GknZq(7;vF=DbQ)Pey>(8pxp~9UTDNHV zWM`gLbk=@oaD*Btzsus5>a}avg7$ksSqUMHNU}L~wH`iT;q zWipFqU+{22lNI=V!>=w3FAVOjL@ey_7_=NN93Ycx%Ae`?L!$wX!pEa_cqmD_-VJB? z#P~>!EiTp*G;_~258HQX<`XP+@~tm}?qqc&hlRVe=r*uqu$9s7f%XAgkn8}x`B2aB z${xyfCyfDe7Tixjc7@lr^CM0c$k%1}Kqpf=egW9asCWX`<_E`g2!m~7bbDbhcE`_h z?*q}6PY_J>*|TScD;?WSL62(xuQKzy#}_*<1XZh8pu3RF0BPK0@;hxO6IlgYz-TF_cdLSkHw zD4$RkjahYO|7|0;Xm*!L(LptnPTcmIR!{f2e+#4&r3Huobx7bn-GD7}@}4*!%xWX^3a zF`$@c+*r9?4wZ@lHG1@O^Of#$@T^(ZJJX3H4fl{mYvfl?aU#9Mdt#On7OXdAcD9zr zuQb0&m{%!ik~Z}x12Lt9Cs!&A;Jfq4l;ob3fsS(<0~6>ROE;u!1#Pmi(5K77^jo|3 zjDdKZDk;uT!4ayz>&7#LJpZpXj@5ii);W5rz{95|!S7=B` z$UaOq-p1U;09`tjIFda$e9wmGtp4v6`$WFi>!HYvUqM1PrdEn3G|T!O=pl#f5yI-J zhZJS4r{{2nrgyhMvxfceWo$BS98?oD{$fztvbkra=DVwGZI~q0@WVE9DdB$k`IXFe}!(IdgYq>zn)8W5+=SP{&m8G2}p9zdplw4u@=K*kb;P(TL zgw-;mMrjO`UsoUdrS*@F_f#KcMuC<6HLUdO8WK;#%YUZuZA3Yd2_@LRx)*y4))74h>aj1VsU`pcGD2fPD?J^AnJq_Y2%0I1`Q0_Y!d z!D__lmfcK{3fsOBAIJauLdO5F-toVeZLjtAp8QYiME~Ee+pntA_CR9#GF1G*n~1 zZ2yMXuO=IR=_K6Zy_VATyFyl1HSW-zp~$}Kpc2vyo_A)A|u zc^F5o=3WUFEctdcyzIMe*D%ubZ!_o6IXaKWetfU6joiYjNRFB*g(=t{-Og7YQAw`+ z=_!c`v3A$$NwWS>6{0l4P(@&O2qRq$-JUMro_~#E_k)3f&NY-tP#rp6MZ~FzZn*1r zydh~gL`L@Kehmj`g>@1*VcnC>sKra6yBOeotfHI zs1-jsD@8o_`>~!ZP{i9FfoE#>6S+G4z>=)Vb&~C5UTXTxziu=6V_=YCTIBB$Tz>M; zLZ}s7IvFAOxvP8E{q6g4`uyu}vp9%?9!_}$d%kV|t>%e;&U0qmi+{jr7a2-+ZhLA- z3jBu#cy>P7huxk%iTuBxnDuP2&cn})F|Og|ODWu9I7@?za=vYYmQ-~Tk--}_wIa!H-u`y@QIMng+h6SVc*jtMcIN}x^-Pv z?qD@x$Jr|d3cA*Efe>TRhYl&<3}$O!)^kueRb-&2uFmHyiUNVti*Y*x2~+WY&tYdT#&=k(g4AD4Zhhrx_?aSS{x*)E60wEn?h zfbuX4(o(B=S_|Rc&UVin8D+8U)`Xb!e=ho5DpbmeuRbV$bll=5DWNSbd2$VgFI;L7 zan|hjIAXH?4iqHQ+KcQKURRE0nU#5C8vg zfu39Qr-v<8nV-@<&6J5&%97i1NKE=afr=zUmYGQP#~G>5{mb61bB1l@Zv@rj&lZ^s zrTXmEO1ePDo)O{k@n%Js7T5i*mZh%jOKu9kC*b+?pL>2Ty`~k;)r(OM2qn+c6!AL0 zR(kIK$dUKY7B|Px+FWXh&rYz(2riJdf;sS4Ys;@7$M@mmnd_xj{HL{}R}X7%ie-q< zW9ZZ9YKS8Sgil;z4h6aW3y=(ImkaGE=m@~nK*N}fS}OQ~Ad zI+g2|?z_=bJl%5AehB{_BZ={;f4t4f#QgI@0{y!e&w2N)3hZWQscA`4_--M5ciU3<_PKA+ zZ-38I_TKxO)Pm}r?jJts_Wxzh;`hv~!+JVo3IXdkT)JFg2rkWR>C&9lp`m)esDF0! z^yzbQ-bbFi(d4oh1qYz(EOhuhnL*6|qHiJb_apuV4eQSedziHFw=6`LixqUwXU=r{ z_ceKK87D?XYE2$qigXAbiNz6Cv0t}lGUehjseoxzlSXR$`=E-=9ayLwwUUF`&j1|S0&Pjm>YG=?5o z82>9}QJ?duxr4h_#zu6=9hyTK)hluXn6(?p7)Q#+&$=v6^2|2FT^uP}$EhY922*(> z9q!nRF7C~+8l_4ZW4zj0_gB;VKGWZeeWv|yiw=K{G~MVFRJt4NE|Za9=NrUWFCIcc znjN@C<`^csFKec_~vjHN|m+s-|!1Us`mI=^@caCw_dBqb{4X$|4@8eYXdSzT>n zCM=hijJD3LdN{15>tvO)A;}YV5}R@0(i;_}A5BrkYfPxdMgrdl5AB>Gj0Y%kWA=o= zzrtu060?ZnA}O7nT)G$9W{x^WuKYMiu1toIM=04*Y2?1pD@0bE?8_ z12>*HKf@Z`R$z$}S92E>vx^dQ)+9Yl_lilUD>>*X<>_&>grc7}rN@2O#Wx$lSr$3! zDc#(fcfJg99;FP8I$7JAdQfK@_dVPE|3Kxf!+KGiO^QPoEyOrIi|4Xv>U3cHXL#vvIz|&bL2SnuTUpU(JwXfWH!K)OmwOrT%c?%uM16?y%q)m(BHk@(M1W+lhr{g>b?j=|e2#ckuUZ zIa+(-G)^XPah#C&fP>eqzRTlGX-!dxMd#aCVuq%&1p&m7g&)|~*8NX7N@?&qc-7L) zx*Ajb52HEymY=9(iD+m1aGx_Ae64#9B5CAuGi>oewf3>?k#6_$*F-P+N|x^=tBGYAf>_D z=30VELFNoPzEuQ=UFgu9Zs%4VwG8)9;4aF)9n?&pw1|Zh>>*jk$+}K9vBZAwfD^+} zYes71Umg8@=R1fVXY`tl>KglE_vN&yKp2BJ9JIC}=5e?VH$6hBtru^ZNAEsALtXwQ z%FM5nBVED4W6k3E39-dm>c>r`Iyur-Q{9AKY~$>vhRTVTW9}oy9k+OYPsB4phE)Ie zk1_uAdLPT$C*l1H#0Okz(GtbS40)5|zQ#ZH_D423nldd%BPi!GjI~v;(r_<2Hny-; zhy^5w+l)be7mX*0J!b~#3a2I-6Y8}5<~G+m zPi&a$Vwp_lla9eAzlX?on$B6q?`DRhpYC_}>!6a+tX=)<#{w~+KCJcL?Q32dQ&VfV zN~>Kf^Ygq1jZP6_Q~O7p#7yshq<<@Zl@J>Xn_xQEHqnXfZiXt($mHj7)YHB=Nja_v>nxZH!@qOQtE1gx zB}dS;-fgqvCPBnJiRy^ovx{c1su&)#oW?ikp$u&Z`4qaf_q>}E&1Mlr3>E?2whkUP2l|xjp=PvfacO+IgqNnE}#J8w-!J>D;5=w|btG2TePiVX`dQ6yp^vuFS~Y{93ViDVO`i$+2Z}7YNFv&x#*cgp{Ye z;RwA$qc90k=v(kt4_twR%VT_#%lJ#LWz)=+-tp5COUnR<1|F?*8kLK_Yg6fq&R%bB z0_NTcBMF~$$~M^w5X@|@*dv_pNa}- zXRk4vqI;Av?(Jsax~P*8zNc{Wt(Ze)d&yP_8~^@t#`F`hX4y=1V)k*JcA_%U)MLsT zpnHP8Sh@LzyCex_D|ovEYTf`s-gjsAyaN0FTP7>KB=nHjePfq<^P>;wES3KIViD~^W=)C{WIgNPN+a- zx)+-I#l5in(zB}%9x8o4*UC3tmH_-OZO&nx%~8m2vN%Mv|IQ*P+{&FrxxLu9y`J(PovXL8CZzm&zHdCsU1htqSd!(o<))nC zOJ{ADMio9mV=(8U;5YUcz3vodtl1_{1h3uP*uMGQ{iA|G5pIs2cxqwf7$*v_w$;d^ z?Yqgs=ovygdSANSkpEc`s?TG3G*o{LGygFgvcLJcrV8aAIBbq;CG`r3{(yI$499k~ z<>W$?iq)D1Ct%FSuGG<0IzKP>Rz2-vk*H9)NRcbyH>yyG#xIIQk0s~5v;43ikUJ(4 zu;kp+PZL`b`M&%nOY}yxZ%I08ck=>&?_m3hQkK!o$mAJ2K52E&%s*+S&6PcW+i_=0 z+u3C>0?kiRAj|XRit9WUf2jZKy*yM@gY5l9qD>eP@;t9EkM~n|!M?XTS)!h&))<9| zimZuNX(e|N?k(M+naU`a@jtOYZvWI>p$y$ubbIh$KqwkLrSLb@@-zO{;G{SPOI}>D zlKL=5>4>?K-`auQ$mG=Q`sK3(XP49#r=_nm?o|t;&c$H=1SdJdq-J@(!%XZu^2$Yy zh};&ZVwcJ2^ghNaljUzV7!RU%AUl=?Vi#Y%sMGKbKRm~IvB7LTWqi*d@X~*OTGSRsLj4UOaj*db?!>Xkqm1FY9f1;kG_g6^vG@WzcLd7fPK&_uex4a~Cf<@C#42`8xZ+mc?v@xCi12$2Mm6Bt+j>eUOMi6OLh zTT_*$s81B7OQRY^e{ zw0!85u1#z;Z!{;2bQrlOqKR&5M7JfK7`Z)I@-QsaZa+z-M&WoRXyE-%R8)or9^G_#G+nLYhf$cm3ct<*4tzVbn`8`P z_VPy5Nvv^~E=q^kVP#}iygqt}7(FCPIcJq8Ly_!8AlHeLwf9DT+HJ|#6*A5>mueBr z@t0L)6bo})eYU32(HZX!95(B~Wcae9#nW3N5uUwR<#kfV~*jy*0ESBmh~h z&Qol7>E)!6eu7TOseRaS4PLA@nwQ%4$=Iz&=TY&V#A?ZsoJt!YQm&I`@UExMPtK{* zz-OX+#2tpGWe`qh^nHpqjHAiaM5S0cu{$1I0p1^`12LO_GlLHZ)5v};hG?lw6)Ih~ z`<9rHp{)D2pN6J?n|G#JvVOsy zFhIp{2^W7r20o_-DO6Jip(HX0$jlg^Ro$IDCcm2~zTLe55&W({g;2-8j6d)>DKM7P zm6Gkx`(R>IVd}Wb-Y%oWV7AUtn(q{so)}AU2Q7NE*?OEkQrH0tWCFE{aeDlY!2=6(4t+6C!pnSPTEQgVcJE4A58e9JsR@ni>NY*31I4Ea?7 zZ8NFwymsyNLRqH~KhxQs<#FIiA-Ajmd^bN};)znT!?)QGvPi@h4~+oxKbd{U4Bmk@ zsE%RS3YxJ_AG)T=*||QtKU$#-*1^x(alm@IhSfuL?5!XF+*bEcw!g6iGcjbWATQIn zb4t$F`U8{2YOhP_kRDMut^!N8G)}8zvD90Jt5tC==x&#^$`kOnXJIucpbdaEnjQ}d zjze#+KI~IVUDgI)j7J(c1xU9C)P^RX_w&291JT{JVPIAiwN!9Nu1?2=6^hAFCO@^d zKt4-3&XB04SJU+6stl4>Q`0$#(*=4TF1!vH!*4>`b1?|T5xpLA;6T9Xu>x>tFwxM~ zdyN54S%%y6ubRD$weqdAA^?Fl`&^eb+9cS!!$>uQw=+8$g=E`P>EH{`UfkOk*B ztZ99W7Gq9reJ@e$a>uz|pCu@iQ#WbEWNCPrybvkY1C*rXwL379fi2(45_`t-wu2D;juzNs>rF7xb1P$33C`&4*Euje*y(c^cZS$QE$9EB~tf^(nn6 z;YwZ>+gbj*zI?cdUVNghNibvCzIVZ>>E*>bI=acFcD1mYVD$I&!6Wo61V_8qWnC+W z$B+xlJ#1S=6z^9ZdEjf7$*}P`PY|WyT)-7_LoY0NEG_%yP$`#SSv_9*hK}fy6uYV; z4kawt=*qRYE{Y*OtyG^o!RaXf8xpkL9xV`y{x-xVIHtWi{*~OKJ3s4-zCOH-vJqs^ zg+B7Udc9eO@nR)*q(XP;B*&w|KK-U?n)3T2x2SBz$Ids`(5cyf*1eNBX3?XU65M`q zVM;Y8$SO?X9rfvx?~9}PELi)X#Kk~VcN5D0S!m1Y)-M}5$X=qIaXnM__RYMh`LSxl z>snU_)^m$0!%Mkdm2j~&KlhjbUwZ$=?dHO0{T~X>zMUg|z9oFYRk_q%Hqtr~voLym zxCy!oZYP$qH*exb@mWi<$)8eVR5q~Qrd#BB0^=(sJ$Rkf%Hta&@5ruy%A|goV^7v- zA?Q6~6&*~i<4Y53G;{wm7RtKxV&@AvaO z_x-wGuh;!N=?4t=?_!Ub)ehp+D^2U`gA6_XFYG{pU;X^qyqDm*-Kk~Cg7-f&%$nWtgQaoOC=9L{j*A$C(^@q;32M-eN%{#F#CMVjCE4hC3Gvyp>jA+Zy znFY75Jq0aoI(TzaZ9+0Q5M_vC=*PrYsaKw&#i$bWeW=L6kfa{#qyJd%Awc%t7dvY z9QZByH|-7(8L{lD@&q)Y({QQmTExbra8YUI_U9jd7d0K#3pZ9xUL^K`_-gl0W>hWNlWn(kdiLNth+j`}3fasW7O(}a@P(l;2TmWb~ZQ(n3w}qnEUejB*KHZB^cUxP| z-+nCnmYcv=_-rB&AyLa_G zQS+$teeF&&o7KtFDgT~K?n|AYc_Z*~bbyZ2v?f*j{8YnPsxK37r+F{NL_|7`y&GUd zw9(Lbzo7{3 z3(;1-S8$Tx`1N5+XR`zj6{{iq+GzexPOAk+n`M6SZ;MjrytsFq74X&pNO=-7UlkM6)t@Q!RDyBQg%mU}g)Q8v znfBnMnVRl-b20+c{tf?PJ{&~lM9=-9JGb`;4lGtvTN*7=XM%g&63@Sb`}mTR`6?;B zm^S}Fmp^hv-oh6}>xU|Eg$G?i8d8fY%fa3~*}>Foe`0r1=aTJT%I4RawTv>rW>8F& zE{Hi79wnoSyB}+zzPU1eNC!$V^%B#H@Exk5Y2;GQZjW1$o0XCiVcxwGgU0@S)+a`} zvdR=+eC=+f4-c)m)5RhGE|i_J0aE_5$880T4AN+Cg%PM)f?*&Ga}bYBL{Zl_9WONM z#0)QZ;*eL^>D?G0QRcikZ6E_8gl)8(ah~g1CH>?wKi?U+?kKIGW)jfrOt5_`;h`o2 zgR}XaMZL@ae&L7_vT5m0!;atmTh=!24dcO?gm8-!w#U;c%O4TFM5MP$0x|C{JUf!+ zH$H9wJKYL4*S%C@tv~x^?%IT`@uJSz?u0ANpmQ_b#kKuA(`iE|=Xn%ZcL5wgxmdpK%AGSsPXszDfP-P?y;-tXS2MCxA; zh|z6XS=b~bkKeY7UjK3AtP z3q;kC54)xSTu`*-AyTUctD_{xn7cf7LSn!K!SnklVE<8z;C0JM1w^dZc7a`R{Ah&} zZhJ4}>2H!jL*h$LP;qlFLKh(PwaebU+dSxXm}(*VuQ5V`dhqz_OkWV2jON#?u?oZR zF?(|CqtryB;@?+M>~xiw&EMJ^WvWF7j|vc@QoKxYxYd}_wS_c4XO*imR6x^YOt;Hi zmS$v4Yb8$?U6gGFh|h`=SRUH;tlp~h?*9^V`>6WyAvHSnhsnL^ADI<_J4MfjV(_!# z|7Jol-T~|w_Y6jTc&1PXNDra(4n(vl5YqyVocinysYjr3M!K6eHC}v|pYLmBn#ma> zI`Fy%nZ-4FzP9ER9%NbvD8<=nc54 zgE&APh17!O-P}wvF!PalG28_zDiqA>2(ew>1?t@P<~d9!2o7R4PPeMK2 zvNBh{!Q(^335H=+JbXt11_JAR?b$teG{H+ibfkMl1(+7Dpot{GcJ z<#!Yr82QKulLhg0J7eum+|T4Rzl_M^8LrT22pbT#)F$o_Ft4CmHEE5|584{m>VU2k zxBIYCdC zr2I4{Hv&oRvWiHS+h^T+&2-RQjO9CzPWJ_+%7+)2K+!KHu@M||Ty+uz3#8l*6PwnJ zQp#t}oq8A*cB{Hjh{vF8S=vSZ1b1FK?61`QRuG(4E1vljSunBG3&1!lg&hQHI&4R8 zM;$8QKfdVP>p-4J-ucuaDkKZ6sPRIaN>r3*V+6M}vIh-cI7f5*rLw;a=*?eqEgq~x z1ISf?Z&xfV_zI6Xe`lFCw#};s);*7Lg#d;&OGMmjWxPAqTXAi9uI1q6dW6m}`Q?=@ zm9$co)`4(0pu;tTM&dFzbO2Nsqaf2In3X?AFg&9!P+AWR6!jkKrxUULJ}Y23aB+A? z_ZlwA;n9wO;AaKofk15KeSYVQ;&=h*E=XMnZ_opZ=QSUu=r$sSfiT!|(QGmhopLx3 zI88*ug6&Xj*Nk53;=alwS`iE_$gb&#rnB*brmAXXk@}s~;Hv_XmadzRH|D(1<$sK0 zzq_v4{bQ*Ga=fn2AGhU-t8xk(X}#udWznNsMg?)JMAy|J7WVzyX1Wb#m(+5Iv5w2} z?898e$UXz36nEnCnM#w}7Wl_1}oq!(qaI(AH*zj1Rw1f;cC*-xtArKs~w->qFA#i~X-9ET1xp>0z z+!y*)lD3J=+R6Uo{9xugx^zO-=x#4rK3ILm{cb)MdR}~wxiP52LHot0K%|JFq8itN zTyYKdO*Ya;Rp}WePq-m=g~Q6A2qH|tm45Eq$g+c8I&pbLa@&W;7+fp%g7n957TE(0C5UAkYd-iEE&J0S-9&RF2B)VvUBMeX>gqYqUmyNr<4To!J) zgOH5OLc5fqrO+l{^VE2gbHDGVR*kP{>G;wCS%3(G$o}r5@8YY+fR8Kc6uOj7iO6?F zvu3@XOh6aCryz<9|sGbK52&N85d1x|ZO>FD_aiJv)I%s$E z>9I?oo3gnKcpSAmpqqc`fzzr>G!Pmp)Q#zEisSr`=gLf)Cfd-6ag&u_BZQ?l2z@5$ z$2K9kr2>i8Sx8Bjcf=k43v2)zi>6w+#*3YfzHD}Amtyi*yW=o+U36EfR<0$3HH8T$ za}L82!id<7{Dzc2y_K+%XQ#cz!;|nU>M}4E#m5gH1V(Zi5}c@m165QQ)p)>3hQA1xA>PHothGzD|!o2P9vDh;GJm94g-Go zDAud&IBD{+!+O3i)y4`&Hog_!t9f5*yODZBr}PrkAmbi={H!Wi>DRTb7w7D}LvKBu zh)(@N3Cpy24R$bMRNb<4Pgtu$)@`uW?C-ZgrRrIhEUO=^Fh9Q!^w?dutYumkm?Mut z(?RnjgiBrr<&-OmvK@edkTehDYPuIt_Ch00!W0NYS3WKsaBAE)|M8;L%lchzC&wo? z)ve)mNVr0_Yq2Ix-RN0Qsxbk#Z4ca|oMAmTN=r~QeTOamrW57E*(z$|E$mA?8kv|l zOSXEbJ6=1Po;V6Bcr>=atPQRqX1Xo%xnv+Z$E=klAS)ekAjrA7HYB!L znbUrLSuL<*7pl8ivs&yNZ!tDY_@1r^x$1hldDh#mX ziVOmXAW0>)<<pmZuTVw(y(ME0Y}2a$OU?OFAQF$ z@iCmA4tk#6lB}9cqiuiYd11QbM3t=d@*qLmKl_5tp8ljls%8njiB9W*rbz2U+pS6z&Gk zO;}Heu-o>msRha5@J875BNvfZ8Y6^8I59REkSRYVHjV>qYzLsQ@AVLH?R^ucIqL*C zQ;PUE@&?F7lkF2!6ldw$UWe#@V1%0EvF!sG{Wg%m=6fF(@sUns{ln9q?}_7oF*U;0 zs^u*g6xPhyTmY4Gv&r0hH)vYOD-z6);xi|+r8vg;7oSnLf{`GqK`wJuVD{ha0`Sk| z0l|!Tn?S`2hyxX$t^R2%*qbH*ULj5&z`CZ3;lhkR$BEcw{+JuSFMB!J&(!i(R1ef# zN)Oh=M*@PP725i+LjMB`%Sp=qkv~XqRDzdUy4X{La#|>7J4r z=0Rb^6cZY*tN#PrX8po`Ko(T^J&Qo}+rJQ|`4ZKcTJsbMOYl2n9TQ6UvJC2k@fiAd06XnL7xh0628v~>{TTY& ztao?bY6dmEuM2Y)lOw*hxce@NEM$|8y`T8|*Z-b~rTG?rIeevNLtYee74q8ax83~@ zkw0L&Ka6=DQHEwjz542&Wr!$)Y~P3u(<Z8g^MRhNslja?EO-sqx*Rc zIABTTUty#EhXxi7J&3m4yKf2Z|N2NI*iPG*$hZ{qWh`1oWn6rRj@-8do`f@S{!EF3 zSM0U~;{E8)jU$#Z&4_u=-XpgAR_IjfzvD%u84CW{Fh}q*{*#F-1W#{$9BT69Kq1-p zy*taW_@7Vq3Q))&lk5ZkVnUL5xnZkkK_1o)yP&u?5hb!nkPYkhUeS+xykA}Ajfptg zTWdRcE3{FYz3nH-NcFs!&h7Rbn%`TKop@Z;&)O^u}o9zi}u{G+xH z_pW%*3-$Nvhc8r=?|xaL_kW!Bd+$7U>G9s~e9b6Q*zNL9#M=8F!}wo5=1)r9w`s$- zPul)xEZBQDjCI0aQ|;fc9lP}3bJ_c%JHubL;{Vs%PF_Q{Qx@A8kCh1ddsjq6H<|X0 z@;w5MY4kqSa_)ERTTWMAZHSkofjy%et#SLSBJG56O&s)=`;Dj9Xj)MTVJoAJhSx`R7IX zTb9W<{r+^|aeL2>pAPR^*)loULK_$5Piku?(Z4sw@Ij&})lwxvxrXfz>1g9WTt2=0 zWQ7euO#!)Pk_k}tKe%(4)&RT;C--Mb*d6aVXlNs9zpx)Y?nPbaZbm#XJZ}MEgT8G5 zn|b^kvZXpO1O+I#S(#<1n#J>bvgTntY=0ZYFm_N~9{X{(q|hP{>XE47DzZe^)jmxqYPAeXL55!LRW(J5vT(ir=xc459iH(5T=3y< zC}c#5+$G){alLxodh>ib@qa-i>XCdxM36QZ`wygWg;XR}_rwmeRk&Q&UsD(bXqyz|mohn=%C(=A6C zj@9C+wi`d$5|<~=BVL)f!mZ~V1HJ;&v|S0Fp&7cp$fWT9tvmNCO5pkid|2(OZH6%{ zY6SlWCIZE*a6a{jKcV&51|)!R$_2gSW_@s@NR_BaWDmSU$=oj~q$G!@z-!ok z450FC_`*;w&Hd)W91_l^He7r7|lYIMOwt?$o`}f*0|+==y=Lz$6z7&n9o{NyKF2W69IYF;JLN4F%t(k_KTaZ z{%OiW=-75xEBR)1yXGc*Z#lW&K_&KFJADjseuFzC73BA^MF<)4YI5+qCGV*0WGW+}2ymgD>u6q`x zdLMX0%cEHs_oc%EZBLG4w_SR2|!R7wdxPsO}2*!S&hFr&F2!=p8JnX ztKP43Mg+9wdcVw?N&D_9QwqQzawb{S%*>3jq-7%Z!)qKTjx|X!{x9`2;5JspuFik@ z9yeVz)702FKcRVm^8KZX%Cr10qA9sPC!P$K4+u0UV4E0@pip*Ncv)FBqu3;%I_b3A z0>&Wo5EnwT5fmpU=HDX1;zr9;TO671ZpAjAOYLenT|4TJpIW};$O`b?m zVd?zA=u5q{e%jM=Vj9200s_ZuC&!{(gFC1+ev72?D!FO#C|X~xmJLuVEvZgw3F&Dv zj9g4!VMmvA_2}sMkYIQ3F(q<&)0J^6_wII?MH)(rN3>m$#*TM!L2MJ|iu|%qdQ@8U$4P3qhJ9CO;f|T5>kc z=_lO5B~U0>)u~09uJS2b?m>dJ%Xl=K(9ZP86MsS59+t6@fSMDHtRj#G- zHm9@AY`ubW7K*$d_!%#)z11vs4!UK=7geo=7Q0!emZlcjW^5+>`T4yE2_=uxR%fxj z8Z*}R>N4xJ&QUJglJN-!g%Kb%v~aiWn57l?nHjV9dU*QuyHz?RysvtmqkhhYvo z^HITN-FNKp989kGsMU+K>^~h#y15U?a+OW(f(V#@g_I(M-*-ee*3QbTAHe0|&-AU- zEKOx(=m?pPPOUwuc{|aq)7>qIe9B->z4$kC*Dhz_Z&=jdeVG2TdSm=UrS16KeHgAJ zm!HJEj$3#1(<*3vh4cM8qr05sQ(YF~r2FV}t#EB&SUXv;Q zRU_~c8!CDxdTGyRa;>4xW^{5y(58VC(AHpz(U}fe?gV+3ucF@#+X}B=Uf!aQS}^{c zP%!Yt!Dc(Husl0Jh4B$JtS(<#(F$`d_o_c%ztEO&6^=;KcN(&qgP-EHY>c9Kl&x}i zI{mU%u!d9L;7f|-i+`3N5%&C;!Gw2IoF(o0&MGsFS&44S{n*#t4~S%a_v-GpJ5o|% z2sUt6_q|1JqTZVDoHj@C9=RR9N3vT`P_{`RJ($DdBZt5bbGN8ZbYdMh99j>{NAlc! z#@#egQDv!u(beHSVLrax154@pcKyTyJHExwt=tN70b#HDQ#O|I6vvP1rT2vQJl6-5NI^q5``G3~( z?v;pws+JD~#9yKHx}WzS3f=qOnt%8~2mzCOYSZ3V;jcM*{>~fdgV9q5>w#DLKVB*d z@{^jmFVZ@uZe*mC(`Is`4@cMk9loudZOew8*&ROgo*0k(Y?9hu^>puJV6Li+jMksg z-M?xI#ea9X_caOL|A5E-+mfXIY8CrmBI>y7-o01H?SIX^c4D)yu4lfeZMc-dl$1AF zi?9)xvA(LPyGp>8yfh3UUi11drtbdK+*SN0TThmF({QO&IdA)l+WT&%V6;4O$>A@S z|8C=Tb5ajEKOlVB-2WYWe~(PP*5s|{JTSN4Jf7g-(5RgQ!tDOb`AG>33(WVg$~vx@gBx zs#knGXvWchRPIMq)bU3a@^dJPKh;uARzXi)+$7~EgPr@Bq<{ARmlpr8ab=X@PE_PYA6B__JX}l} zkV7$QLOcFo1u+@z{U@y`nM(|29{-=Y?Qar~HrZThW%@G-=iXO-Fh%{T!Icbt3r5Jw z*Ih&VD0df`7IZU}9e#+@X<4`AI_Fd$)21NE6(XY6OzP!%-)=uj7(D+yZoPXUIh2Cm z?gPKd*fhbu>b;)QTlsQCl-?=d5jjh*arX-#goQSjefNf==v zd?TNDB-4p+h&-<3cwm2$rzB)$U6WA;{P1U942F=L!+c=ucd@t6K_Dd&Ys7_|QK~w{ z?Z+Nw?6ssI*N_-_1fK={(KZxS`v`$@h&aPuHy3uIidAY-vnfNbD0@ ztA=zr7|r0j-ftB(qlo+DWcw^cL3dX1&WlfU7sRFPQ4_!Us(Y5H?chgexneD!7F4&(XUL=lx$$Dsy85E1OXQ6 zns3VET93}3?lS0m4={|}k1FA2_9|=H6P6pRQwMMU`hCc|U#tL*{?dF&@1kw}rxPFEx*_J;YB;qYcS1Tllb6UUx)Ps1FUn;_qY zAqXlzw-drD16xqD4&R0UODuoN0P@qIW|ibX+^qdbg_vBV25ANFiMPttTLgYW4^;4o*ql7Z>$hxRS=zBSNhwt zC&XFlBeUqQ>&==2%5686eg{s_nSO`^IMNP4d$eIaM=PtNYUl^-hKSMlMR4?KU44Gf!Xl%W7oJwit4uCy5+3x^?&&bhocwfOUhlLnWh5HkRWbalXYd<4 zVeY|1W1cQoN7DVuDHfvmeM&5_r|e}m?B#j0SA0DXNhwJ;1~k|G7?WWe&fRFI!cybs zHbY4^(3FN51{D0~u^SpbOO1X>#4GLE+g4hD`i+ANYHO=Bsv!>_(SIwx2#58&7_={d<*g z;f9erM4Gf9%gpPOD5%$BVo2fo`kmSZCoNzehrQxOVp+g%R0%xA<&IEpDq+Vv*{e3) z?>aOAwqE25cU&1y_z_%S+9tj8oJ4UJ0AZZJJ=JQoor+w;Cq7BN2tj@4QYkNQ{c4zC> z*X7}e>gRof8i23L&D{XVSsdubeCKLLvxO8MdCt%1JV+_)GMx7!@2!bJTESbB_?FAw z6L6zXy(G?+X5SH=ofuu7_0Jft_PsDcDd0;jqLgfWflAc%OZGIq>Gs1XSMzck4I^zx zh|kUZ&?<}sV4HVw=um-vfY`vpDMC?>95W1<`p9-JT$=BGdamgz-DQTpJh^B7kDq#1 zmMmr_lKmuXE_a5@4csU`z@<|v>#4t4N4X^zGC6o_cIm6_ddemJ<+l^}<4Be(_=C^e z0WwUIiXlZH;i;1rl%2UtIpXa2v`6%uKswvWxT~Hnz3;p&z^+5dl0C+4FS%rhuPeZGHI^*}puP^tiV1r!}?y zb_Kuha>bB(2+@1V#nG*YFoKLMbr#sjlAl$bNO4;;XbA)V#1X~csR$d=RLr#Wn)02m zv-Z^Q_rMHH%5*J#$o@QLYcfPF=d--l&2`qvO?!4S7|OzAi5I;-n~M;tZP98&(Eb3G zLYRoL2Rv?LGp(qsQ#)`Xiayr8dDg+Ei;xqx$logQ{mC5S(f}#3mx=} zer%rH*bN-LdZI21=7F{KEZ?8UxA=_~&%(nl(@! z89bd1<^*pDe%JC76AS`3wgb;jO*=|I%iPqCcq3OAIW4xTC+T(0$$5wXm5YODe&HjV z@;O1J0gE&P)Dem9BcJxbU(!axy@(X8u)Z%HscObBn}V_?E{j4XnP=kQ^w#Y=m_U+_ zkx6He(FlF9@3d%!TM3ARxvTq`SMKJ`n*hP#UQ}d;t9^Lj#N3736?ojHgs78kTc&WH zdL3oKPs%C6*fek)*Mq|c9Od3((L+a5VWe^_9$)0s%EodC_=&?JZTk>7Ttzp0VokSj z&9`@@bxUWd7aCSa9?`p6&fDoCZ++a;IzS)N*QZ^u95Pm-mN0*Um9tB`>(zVQNvaHn zKXr^*x0qr|*nSYS)8KSb`MG#zeYI{%9wVRxkIqb{*vHt*zQ zB_LVc&IHScz6Weh1Fh(G(G7(N6bZ-OR>%!hgh9{U9j0od0d-V0x!!bptM}+IDhC#^ zPE<5xI7S)TUTM?5Pao*Qi9ZVwg0h|EhO1rma6>fB#V9CTq#inFNRo@1U14|UKr|Dvfn zu6-bybzkGEZE$~ih@uT$0NXaE$IAs@74FRdwg`xYKs@~`Kwi$LC@B+Q^KWG|BgPhdPII%WAY(PU%mN!1^|FONq ziM%82VcV24k;--c{ui;Xc5vpS2YfL16KDUw$*%Q?S5e{DwPrK4pLjWXl5F{GPshu- z>a{(v?|auSloAi?`Bz&uEJxm8o&F#~$5N-3&j>=I&97}q%yHG9=!Wc6ls+qaV@&6e zmUJG^NHLMp!+ju3kpO2nTkH@MxqTz}cbrI5m9s}po1j?W;XuE@dx)Z^h918=M&1Ur z$8tNjTPrf*7d*d0o>v1xARrFOjnf-ji+(hs8L$i+I58T)By7+Wb2w{9wj~q2ugb*t zw-#n+J5NHRMhd|M#Zx+T)HHN##<%i=mN8oDkc5FWeqSRBQb3{dOSd;nT{gNC{{RgJ z8Aq3Lwn-a%>4wPl&)xAb7mcB|a$K((qj-{Yp%(zBKKr?U(yu^#`1JxUwmH%V385iU z_w5N(tEk4D2fSbNU0o(jbcywt>y`UeQ>42R94+uw z4gGq+dZ?Sz5h?wr1Sw50n)kEH^k6J8$gd~*C`dzAQZN&t91$Mrs4TOykyOtsu(*)> zxWCsB_b%12=~>j(MYErlrn(xad>tpT%{ye?0Is6L);GV#Bh#YWfC&3d(SEk$D@7_V zV3F>e5^uirP6<%~lTS4+(>?io!LW*8-kx=8@RG+k)6e$A=X4iD-^nIyUa0F8pWkk_ zX{`}PSXjeGKVtNl#VIK{i?iQG+nxw#Zsq{&R+On6&8 zoP}mC!@!u0&dXoN^qY>Qx`(+e1`c~D=b^|Ld zM(k=sOD69&D$|cG2-nq{d#CJ2!{d~abY*+J$<1V|Fxzo!US^|aQq}9#fsb2GmR}Jr z4hUMuUlYc)HbSCwV^+US{^az|4;;P*K$RkqwNuZD;P%Zg$E?caG6VIKoa8al{#paN zBWUW70!V+}d1+w}24f)lB}rA*%_C4=juaRLQj zCNOTl__!2f2BfRSjo6v5n6C=hW4H7^@v1`wH4fNwe8eGBRipn149Sc1<3uT`M0?mp z8!aU+P>rx2LCw%soEaFXcS~SrfDkHVJN^g-`tu;`kH&mFfz&k5x>B<{r*wW@CH}qR%Delpsd@t^Y2lUcE3>IcDj4zRs8M*K4HGySk~b1hpfW_* zMaZF6!kKv5nFu%f^Ntu8ekK{DXi^kiWF~!29cTEsfK}k?-Di!4tdH9ahcyBFwaiPs z$xk!OkA~tP!#&f~)n80Z5~hX9Dtaa`5U2Wr;6#75&Vs-3!XkPI>QzA5EyGt#T4RSh zAY5WMxpzrk7R(v=1HF*$Yo}WpwRE+Zuky6Ru8g#rl#xMk6 zgIB&KZZzL@utBN z_`;@YV5e?2vhkaU{CVzgKe8u`nS&xx%51?LYW}(|yfC%_af_j$Qqi4Nkzfonhw+Wy z@zVXcz5D@=PqN+0PY+*iOc@DVoTRq6*ZF*mnKL>WwaMW~w|c-KCUMCY(Qb zw@1lO^9XPA7Gz~|+0gMkEu?e$Wvs7vFA9guyWX1EM}zi*(E_CB&#E)dQJ*zy8_zzI z`=t9x!8(`b$xKO-UmY2@pOLOlAF~^8uhM6@Fz20^HK)@*1IB;(dQQ@$d=N*Ls!it| zm*!al02dt*KH{_h|ChY=3CQ2h4BZ58X0dp7w9aM#xryGw%}o7!kxO@r?8s3aIv=F` zX%aqvmXmU~hceexZcs`j;Zx$r5Ug|S~lgmVds90h{~%)nX;<@@uAVW7VDaT3(;ISUvz zMbs6)rz0pOPvHu1=Azu(4_KuAU?rI5$N^d%xhlFGC+6(tSyPtEn1^qEh@sxJ`Y@AE z=6yTL<*UwYnx?Koq{br6gjHR~pY*tzx@4%*O zWu{)vVmt!PY73sVuPV#)|1F+$RkSQvk57zf=T;O;sj3rgdKggiZ+7a6hd%PlTxNYc z-s-bduN>N39j9m8$+)lYs`u+NNn`<}pvkVH_d+=95XR(1T6^RV3w%`6^j6{BLHnw& zRhrpm)*s3;J4`x~u4Ax=V(D79OVIDVO39~i`nGCxq9~7I6k59BQkeRWv2z(5M=C2w zsd$vU$!K}5t%y}rNE|^_3sZHjHmI6jkh;n$-)ow21lmOR&?i_pll?(*OQQh=-~7%1 zph1+1OCt4C#+ytaA3_gB`|sdtN7E5ui+&G6kvJabB1fX9r_3^=IMn=H#%9MZamCT* z=kv_RQ$SdmSeflyf<$!<^bG`fu=0745p}5g!V#j{*U`sR%(YGz7>`c95M5q6yFA`f zIraPxWFb;|<+4Lq1!sAIWd7<@x7^1aJf@x#b^rTS47280z~;ym)F zy2h-X06@%IYT)zgY=&m4+?nBwMy?KrkyDf?9VhkHD>;^jtU7F`*XuAvd8r`|q}ii_ z5zvYI1@sp@dbTv=PdF+olfZaJLH0K==w2p}izm636nCWLq*i`No%&1z#GP7BmKFug zSw^V8Y6qfnLdSeY5Z$KtK;hun-O8iOc9o=^$_*G1M9&NlXOl_jaCb&DzW&#%OftY? zR;V6-7g7M6rfh(wbiNAk7KS0i6h&e<^WJ>o6(*#Cpy4KLs)Ww?i!WYnYWncC z7ULE)Os8h&ppfWCyQ8foK9CI0nTcM2J)Gk%sPsrWE18){P~)&%rJ`%97s&*?6A%a3 zBPnIg+PZ@)S((`Dz+e8H)wj}{%%RV9d&|LPeI(@DzvQqnXgkcT{+-@aC(PTsW^I`7 zeNGdenbdtEpixUcnU8Po=J>p!ccvy4vWFeMMXsQKK$%v^ZocbgfxwDo-|esFy>{%+ z?-4dg`#Kfl6CG)}R<>g#^J(P-r}9hJJr7TB+a_YKmu}7o%d8LedD!GXb9pc-NKD+z z^R(-#OzS&?_by^hr%y&Fb+&BucX>1YoE|39CItkCJ3B>2LiCDOK`>#=uwE^=pKvh~ zB08{80SM*ECV1p_Y#R+h}Aj+X1bA3ouly9EgFa%ir+HS~Oc@QXh zNh~uSU#?lBSqoY~6Umj@GenBm6-T~Ux>(Ry0J*SVh{&H#_~z}`z6(XKJ;r2}YZ`Ul z(dZvP_Ag{p>;8(dvnrXohl1kbtfoUVF-s1O5MHSSt}2lGI3{&PXK`~i=@UQF6QCh8 zUuU?Px?>-zsN?Cn;f~rL@HUc zm>peah0o1!+=fblg-(GB&cz^bYlpOcgI=KeJ-$b50H%T~euO-j~&>Ux=lc;;g5* zs<1XjW!_udtsjx$pOff@5hpCcrj@j;mSXF_&$ul5j%8gRQ=7G^(<;xhDsfzHPmtGw z!<4n|r)%j2;SU8>`8>1ob?!;uE8VdBUD26dOq5M{6 zgrkAAWs*SNJwpr}szAu?(tk&??)o_}?hLa+;A9_#CC6%Urp=h%IqC)8;l51Fwx-uGVZN0HR%;Loi(~WkKW#3zUnAKU z0SbI+PQ3z6fY2y8-eA8RnK!}4~+LB>A%;G6$o}G@}*jILjGjh z3=}?2zy1u$jFs8`bi3J%l^%{4OI^n&d!{2VU9`|URCPpH60==RO}9g!JjXx~V3BcJ z+3Ll+Hl`1ZsRh12$HmHGoEi|2KMB3i?R{(N8IGjx4^Mx5Cw6$rWQ#qg?x51~v;Pp= zgRTT9DhR9r!`bOD|o(F{PBs@u&#A{rST zO##>}a_uGS+rM&k8yth|veG?s+F!0GEQweO7L7KM#F0Y63Z;}p;caiS6xGyVO-ZQ> zo0!cR+Zd0&Rh){A#vG^Njl)+H~g6fBqBig*X5l=|<_nWZ8l`MTPxdWsgL zB=%aXXKyu%I7z)!OZ6b}owgDP{0MdBSX<%^pPAB?G)vnJmDgdCJlM}6hA*Zz@Eo=} zS+83vpS(9Kc-7ufcGLRZM6X_qFV(s3*$Lb%%ukgm(W1m>ih0Jpox@^{u+uo@s?{wA zu>E8h08dJNaR~e(@1}9;K-U|~3^;9uZ1hywWvBU!9VI5lL#|0x9^kT9H>M*SA{Zr?Tko+pAsVud;~PxlkD%}k=7<&13MXrImYv7^+4hk`7o6(JNH_P(WPYFN?W1~9btRIf`l93W>>od zk^Ey#uJ-*6HbVR4nyx>xfMj}6CRMGHVy}Yg@Edy-QOUffbCK?uq`QB;lAmVEUOtWT zlT(yygLB&3vto8EWsVUg^<`kT9VHQC&r#L`auV(}Wtm(ap5{~#(;cDu69SZTd`DY< zirNZy(q-c(iWcTR%*dtQt^p)0d3cyh7LO==66oe=rn=BkJ^g7XOh7kaX=*VCI0bRh zmG5m9n+4|mX}}i%8&Ggp$)Wk*Av*MU@YB`@O;p&tXD+cJdY?0@-jANU;0BS$gtDb? zR#1Goe$nDjW$DWO=-=zXnc6RZiLF-GY9^s?C_Ow0n?aQEqft8xWdr}-JhUZ*#eu?t z$U&zXr;%z~j8J$=Hvid}kxRuD4v?N6qR^2q9K+PuAF5<5ilVk`hBrX#(e;fouzgSf z0^qBMHp?JB)GmifNdDdVzRY%WWoB``D=z&lvi;~F)zFD8KM0zb#+<{45uT%E1e-Vg zs(X=V^yN*dq|OK4zYyq8h&*3l?lR-sF}b6o#^|G!)mY``rp1L3=uiKcnK(BC8JD%A zz$-vqg+hAHVVI^K;ItF@2{#2a78oT>f?CQ!ueTD(3!_X1Z?N_Df8?{eeo zhIpZKNk#|@Jf{an2>l{J=M;_{-j~&Z!wO)O^zx>dkT4@5Ywc>e9Qe^j1{kTUz*PhO zhFqBXJas5Y-R(HdxfvzoJ>LY~I=hbhv+;7o-kLq4r2E%Ybgd&1jhfh;#VoF`HKAs& zAUkl$RsnQg?vY1kx?em^=ZvELWb;cGlirNj;=XSAX<-7;OMIDQ<~sr{=1!1_s-Ne) z8Fh5|DLairYw0<=Y9}fhAlyJ0-+QGHVRW*p3blWPraY!%^W#{>FBpklNVCUY~f5m5#1`TxUcDgb(S_codulu3< zisysBtG>@nQ%iMRnHjU6jt}KlqCj>kiGFfeWp4I4qW7}fm&E2^hyS=emwX6)L2N8{w*Rm{hgW8MXIBMfvLo zb@P!S@ZkB32<4eNiE>`c`ZaSvX<+|1FT$2xpp6xNr)plbl_og>m@r;0A$0 z#iDnrThO4`)o|1y$}i{0CZ+CC*8dK_xtF6~v!WbP@T>{5OjyO?0YUIM!Okf3NKG!} zCbn|~s}SL4juIqtXMKZgr=qTo<&5xUuyCw4<~@;KIE%?2d=%y1z^Q+}ke$X)yDfus z?Z1O)+GqrUI&G3Ooc#wjJEDXq z$Brp_S~N4&`g6<%ye~*RacxTsQ7r#8crd~NT()d0pF!|>Z?>41W9e8uae7Si^Aycu z#ZPyP$zMMh99B{L;c{(#J^S)HTD;eM1va>dZw5^oA%0csZ|m2bg0oi24$Fu0UH8a= zYDehv^ZGFxy{$SCBvOH~*JOIbmx&GsQo8=SH$9E6KU-HhKah<6HH`EEQ_EWoZ|R-- zTq{Y{^w-Ddn@S?xhJNwKmqe7Q!ij3Tw1ztiYN|^&VUe8XZYQ9Drz<=}M6M}%nWekd zr7BIdYi#k#R3?<~^_)kpse)NFG3v8XnmE6HzTyriV&(cCw}l}b zHo<`I^7VN=!@XuM6(TpGAj;oXzk>z4oDh#yq8ng_-22z#~mne0iZ)!TKXNo3<^pWsUPoXky+K`r5YM2LZRl&m?0**>iHjEKj1VA z`z5=q_CG^vg4C%xE=)U=`E??VL$|$cXG;hTk-J{&tHf|&)J#wMD@*qKKq2eMggc9^ z0v;+UG4$-rP;3TPqy1`b40f0?8NjUW?e_yhK8`8;V1fTW|1JC3*bhpY-_}v73YLp= ze7cX)HNR>eBBaC)io8P)jL)B;0Derm+W-ET!n8}|EU3Lw=k=tCn05awqc-;SC=3|({vpDC@jX5pR40H&O}E0F zjKgPQ2BugJ5XmYfD$wQ`wcN79zy73yY>dPj>M^8yPMx{s7g+Ura`+!waz32t?C0aj zec5Y`7<_`%&rwXcQ-k|Z%6~20Ivv=g;v+P}Qa-$X;eswH*@QOG8$#BONGY{2{ZA=4 zy+;C8L2yC{6#kHJ2HJ^_j%~o(^91Mh3#dTGB`uxRwKJpFd8&;S)ySTtO?R6;6dSqc zutS^!dC#^zJHUy39|TQ{F~@M}nw1vE#~baJ${S%lz+cfOwqq9C`PNt6Q)Gke2WYSo2+b5mt#NOE>4iQ+U4Unw#JKQTfmV3i z;lrTCP;&+|OzZ9J(I|_Uu05mOHo*~XN1>DdB=6KUpJU_?K>C+rZiiSeD`uquvhyWS z-N0L3Y$t;NvGVepU<_y-evxrKev9D=tPHRW{Nnv~^(P07b~C6tQde8-=z)P`f!jGI zbHzeaJx(7sl;8T99-hobBM7y_2ADWASF;91U~uv*u~N#etG^)#^A2vGas+!*fzkrB zn%D`9d_u%P_=ayWa4uLtg(62k#cwqm{7SIvS!%&7fmtkY*Rt@$%Lh@9a60~v{XoQN zPd7bB&@%}uUBjr*3OZ{4Bqoy!=qWIbMU+0?`kB*IJrYnf$psud^xn0v7slpY{QF#p zCeZ3E4d%zE1!GArXue61T{Wfi*z-x2J=Aquq4M$VG0SJzGiFln=;~m(wywQoM4Dd! zhsI4uIj7wA_GjfF1n#-UQUMddy61Ihy)5*6sjwG&H2Ox0^+;_Ho8h^aDGkd{e>_lB zyzfsZS`YiHfz^lNhLwctn1}D|<~E--k$cK4Fcnnd7G~YV8~y_&PWXA|NFFCmBqO9; z#Nskfpw0pcKx|_BwdtPU%sXZmDPdlfH;IrpBg`9u~`ig986~m zS`Wy4+{N!RbV}+u-e_1w;fpgEeuH2tg?%7n@mNHSf-?S=C-gS}kHD1#`CJb}e7p9~hpO7oGi#tfnw1 zbE-?&UX)biokbkdVtrrp?rJ!KWp2^|yh#x+>rVeeJV^iCr4&lX@X?afv)R3ls?e{7 z+$y}(JLLKM<3h&jIaWTUrLi|eEPl<_?VHat>A2w}`{y0ehk;l?Jf)HqCH_A2v5gTu zPNadd>lX{a`)5rOSPcd4T#V_Z{zF?>-KZnAx-UUTgjGJkMorV_$PX$F}VcuG7HEQjpfv+3BDy0RI>p zM0o(p)!EyMC-I;OfN;?ZZ+mhk4awVO1gECr0cnz-K1Z8rp6Qg=sKR<`zzF^Tpl7gw z6Z&+Jl^gdx#AP4W3{2XyfQ3B6}(AyBwq_K6)k|9L-9R_ZUxX( zY&@ws8gzR3iyl=LDoaw1{_A~YAXh*gN+AXm+r)SuA}IQI%Q8$nwMm}r$B!S&=**`& zdBq#LtgUfgv>lQH+j~DL7~3z=vX>NZT6u(m*O7CI=_9&BBERL7?=i_a(Ev^jLR+-5 z5cm*Q#(S@PpT~zu=a}9GnT@4Iu0KqQ5iu`L@um^fyl{|N9E#aWkAs`35DH6>j&0+( zaqagGJm$|`s@h}HOVMtlrtKiGLwOa9G2TXKR2ULS7h-Ue^| zhAD^$IpFk~8U_;@%L1f+;W> zY>ko!4OXk#?w7S;O;}}|;pJ=|oQ89#@tWdmFc~^h1TNqIA)tqx|>5 zRT{;Ec095h@q(%+lF1tKiIiv;6{FxwdxP4%3U&kCOn3(sKv}`&N`50kb_x z#fSdf{nj)DPrrOsKlk{j4C)(QVM`YBE@g59olvOKTlfh$YkZJ^09-X%%h5w@4vjG% z%)mjzZQg>VZnRSdhYh7( zGz$i>%u;R`y^|?InXhAll2kh)uLjX?D>+cm8}0sLsf|GjeeLgu?Qw z`gbQ77N7{-oMDvqG=ISdk8CZv0LB2}M(Iv(g&$+LB(&6p>!M;ONmfAsd#I-|V9)mZ z?2U$^DuR2$szP@NtUU=I0mfpz+)Lj1c!-iw?f^Dm_QsOIJ=7N^A z-RbtyDZ^g?a#w+HqF25r_2yUBR>}gu(#c_Aj{j)kb_i>l%XYDAO(3R`rFZi9Hqv30 zBK82ma`b<>Bfagi2}&$@u7t?Nhv^3xjb}uR)HCHw@+c92;^Ls2B~hpFO^&Cw&P065 zKY7d=`4G$$d^^PbdR1vBIhQlZ#=t@Jw?Ya4nSmXRPI=lDpGX+GCv~+~QidR=Jyl^Z zH^W!#lxHZ`kg@J!x@Ki0FvZrYpLqr+C@Eyx+P?_~_oRJTrrAZ?Jo<c6>u>>orRoEfbf+F;Xg&n9gD#UC_?>W zCbrQHXG{yQ=`K>xYg*2dUjqx~2N**iPJm}Y0W{&zr!YA{@(!#=R0s$7(o?c_rl|OM zMP?B_v+ruG0fLTGM8{`v#NFz1{wYEb!+#kTUM{e|NI|m7N0W`vXB`(+**G-0D8K&h7G*WJ|hq*`@aS^yaC{EXh}u!aSOkudi;JqQ*@r3`%&Vt zH!RH2x=ZK2;!u18+zv(<5J1uS%s2N*BEow|k}-f#=x ze>Gtu?|+BV=JGqXh3WiCV&eEDSb)v(gOkL`D()dCfX0mc%@{60?XeaV0HB0F22PVv zus==8Vb(cbJ~xAkn^+N0$hH$c9ox`Rxo4peq%@~W3u5-Ud5!!yxuqOgfWZH7Oa04$ z0}!twyKMr?9bnT};6zl{W3kcqgD(T117dwmEDE7OQG!ay;zNf3jIB`!89{9OZVd{y zih#}4p~G3jN=P76sh(#e+h5_y)xup+SUKT?zcUevr%<^J)!s_%#{rViN+sy|al4B|hc&w}Xp=I$$`3v8X1T^4vM&w;y*iwGrhl*ol^^#{rm)Peq<&g1Ym%aaLQ@Hy?yM z)HV5zSXfZXhxw%p0S7TDsSiLhp2c-IQOG`GFe32#kERa(VZ^?q%)6$B-Ly=BnFfhB zK$y0wQBj5sw7oiyPfIb`|L-;Sts0aMNiqyS!R&f-QLxi(HMc(a`b>F86;;FXA zy}f&OebaUHz@b{TeZ+nT;#m;n)nV(U`)P$|U7`PCFD|F2w+~S6xlmV4mArlNIRrpr z-e2?*!Q~r3xdriP;O1$0ENkN@+_o>yk*)D3pcKQ-0L&_Z$w2R6E_2s(y7; zH6nV`JfXhpe})bz?8uzS+z7FS zn)>pVVL)l$hh_t=+;!r4=oY;nJn$8ukLHSge|6N_>n;+^Q~(SL_;^4kfMPuC1R(q3 z#S3EMKccc|*t3F&-#7s9S#a^!m$Y8LKlJT&WNH3WOWevMRE|>)LCF`L&3(T;qzx#eL4Kx? zB}n%f+25J92^dW@gUf0;z}R=&uyl_Lyl|JvPls&ti$m+SQBo9P>0^5FQ7_>=V97owO_3oz!_w%GP)-8 z31XwEdO9N~UM+vUgDog`{wQnoD@+I=!aQbwUYC1QCoE@!Gyoxb6c}+R1$qubgp>{# z=faB5uvV=u8X>S)0p%>sp)nhCR$%FmzmWwQZsZ#@XTsDKrh!iLH-_%1#`$Feof|9Hyb)$w8G#>W)pM7xTE zL?;ED%X~z^uAn+V)-fH={_|?xQ&t6~AIB+4=^#2rkj)?W$9=Q(LpubW;iF*vD=b%> zHvPJB6y1aGM*xqnElsuLSSybgZ3;St28SBq)~H(y`ro8~2T0Ll=c4XBjaC-Tc%kn}v^1>< z&}yL?mib>eK;6cjsE})dHv>dZS+76XV{&yoPDz@cEC70&h8-<6-?y9bRP2{)V{8Gx zT@T==bU?i#Sd!U^&2ZAtoZk;l#g?mURcuFHLj&I-R-z2rB*x8yfhYF3S(q$3) z9GZ-4XZ+ORW>$xD6A;#|iTPW$7AV+k3L7|>pYZM}+!G$302qhuf_@Q~{;%mbO!^&C zPuk{#KQcof!Htw&F%zTFgfTMYGA3sP-}no+i*H|rG<~ztljqBJ8ome=5tK+ru8W|Y ziLOEtY*%}8b!X0*TnvY~RbI;GM58x#MHZ%h(W_Rh2)2*xLWJ%~2i)Q&KYj6y55hmMb#9q^UjlF(H2Elaoq(62*Bx@Q>sI@?(~1qV z=BCxhqga@ujnM*{7sxe}TS|u8ZjMpmX2_R~XuFDIj;h$TizDuWcmu!_B`E~<$=E&# z6O8y#6Z{^_P%yVpR0WOBIh(}NJ{hSDjT^-oa<0;VhpMvVxllPxB#Mdtm8yF|8~`9% zWP#)b%3)I<$BU1TlW>#UlK~t^4WY51yKFgKR6J&Gk|@2jz7Th$-#i?QgL}SntZK&l zKgIZ!1ZydXh377|ysiP*3jp;%HBr(WIVbk(5XUc0<8qs{`{ZK^IKha}!0KcjfgTC< zo-XKI%u*foI!MNJoYX3iV_e!N5|UpVflE9i0fr|-VlF^cI6I{?^$xf9{3N8+Og)+r zO=<7Xtq~Kw&nI+-x~LpLr6DXwbPAxZl0QuG9qZn^*^8}23%r-et@JAk{e_WNp5ys% z>=7iQVn$m0#UDT@ja%!`D^>f;YGSua(qaUxr~pG4fnWsFM%@q2Y8>Ep{?MDwKL@L+ zd3op(-K)&a_D!=#A5BeDXNJo2(Tf@A9PcLJjOVC6v~P^op3}*yb%a?V%6D~YW#Yv7 zXXEb#$Ei22XJjhned?to=s@qbm&{stPoOV_8H z{K<}|tvUx}JCgcdw>c30+>D22b>h>coo@4U?;b7pPeA9%wQ`z}59b;Dd4nZn>3atk z!tO@Kgt5%3LnPDfY!bRwdTy&8FmW4Z<&sXWrfZ#D^(s^K3E1cwo4B8)ms5xG#!~jp zjUU!DTXR}9E-8F(sH@wt7!X)zyI#NrFgz*%U1Uz54b-jETS(UG9x^H8KZNYtvPzbL zxnoF6eySXQrV-1W*%+Mk(dv4+Va3ow+9#miXnIioSfgELFG`sMF()P%*PqxT0__k~Iq?)pVF22~VwPahqgg5% zRvQv!p3LN5zvhLSPj?i4fuc+5&%cLHvQkw~cBEg)@Cp;m(Faf5pXW1nplaWCY&mlS zY#IVS<33ElGO@I-JLL`gYgxr?c>md37Dm%rSaDGM;`Bx)C0o$+^(hzvi=GeKgF;&X zUOz=^18{*W&)MK_T0<^?#+3FrS;O7(#nH+CYBcQb@yS~#5^f&6MMRm0X0wj8z@!P z_>Q4p&RqwC?GLZ#2yv^^O9QlP7pDg!K6zKd;&27UkuU_Z{}dyIZPF0jGa$+PJo#?s z@=Ns|E#mpb7k+@WHGU=uMh>QP0peq zF=OeOl;2v<_Viq*n~T*deDSo#HvMPr0dmsy@`XvQ0mHw*-xzC&)^5+zHp;g6vJ*BE zBaaQi@}C1_zMwm$<+PZNx*&-8Kaw^+#wq~h7i^cA;0p9$@L-cNI%@Us++;!i3Z2Zp z7FiY#+VfB>1iCb0^P^A^>0ll3+DJ-;qN?|>de(CFI6w4AP-O(G6R8_Ax3als&~-cB zU+z_@()R9pzL8!*|-$4OaPWz<6eqD!sa2$nd`F-8mF?L9Z>pQ%rCouN@E zmwH8*`?nSVlwpDEpUtEv9`szEp9s80DCbQtZI&yoTd>)nlA)S}I4rAwmSwxBC^i46 zRBxpeFMqZuWDqMHZ7XZdoa$;8YkjuW-DQk(7y~ti4-X(32wG@L75i*lnv=QsyY$49 zbXCbj%OR~zwHS^D_oVd5A}m-D7S*HEv;C)=`BWj4LoXAwq7h7X_6AT40A&>9fS;9L zsGX!XlX}LaM+-t-08o|1h@`|!v$@{FVvf$%`<9|m9!I+5v#@DD-dRxl9w5nA9oNP! z#DZpzzDJho$&h&v$JU};PcUNqTaG+)aBhS5gxMNLt9@Qto+ZyTFTcP13ddU;G#?N6 zLFGR~_r=@>VUg&;Eot&^*b&Q0u84&>9Y@5s1-^-fBxSOu;%Nr9Nm)G9LaxLU*|`f1ept*Xp=nI*`!5HdEZm zxP(z$6zIG%E$7O1$2%14ftX)TqgEfQk83U`&2jD}~p zzO76ej{~2nWmNt&a(u@64GIW(q>U*_rwmEBc&3QunNso)XqI4pu7KUNq2*`jUC;* z;W7;oL{oZ)pysWxldK_2r!*G_U-*H)?ko`EgnLl#SxC+Z~+%$r@EKz3m2EJRs;peK!=P8|4ah z=HnB6kFU#5aUwKZc@y;(J=Zw(%xxUqf4+wD{Eic1w=6D$VoU6!HqeZ z7Ne@s$6UEI)xe6ugu9kLDwv_9+$Y181~wQyQI=<834pL9#IqBGo(-JJ#7dWWe*84K zspJ4eD9Z#cKysnDP3x7<(q;!s?--kMFK)1RGv9bm9{@V5RS=o&s&}exfffI>*UcWX zcJfo)020O3EVZZ58_#?j66M z0ARwh15SsnWnrHcZ{Urdhk%)chCJqT!G`11^>N8N<&r%bjPk=`(yD3hjNg+q^A8N# zk%-$3We2*yH|TGPYl!L<)S8ugiL+PAm8y^Vsg~yM)1r~D+Sn*XWn!+So)p$?ToC@W zFKnNE!G8B@{vwj_M~^#F7obTr3QQdT<~o;|OkUoo0!$2+&6VE4l!pdZ7+>heA*;@6 zHYx6ca@*%a<80Ly(!|2AQ-JM?!06B*8xzmO1`zU()3cDjEWny%%4rlD<$BQ-ezBgk z@>XJ2dzKgz6UxcQur$)FeAUsZ-}*N{uJ5vta>QavgQA*Kw`C+VAX4|v-_D40&=2o5u%H9i&?F%?vgyyUhAT8 zo=lf12tkt8ocX`qJ=z3dez^WbifT$kwDAm{aXbD>G%SZUOhVdHZ3)6~2Ox6#Y<=!~ z{zpaPqwxpVTv>l2fa6Y4U%R(&iuxNs!P6j1ZO=AJ=-BX&L}^8`%7e~!&R<6PS?ngZ z&11ktL+m^UItos^iT&Qivp#(HS39y#xvUTi$*}AFG~IYwbmj}+UdciW)}^S@RUic# zkGD`8g91<=_8o7V)q^A@sJmM`)r>C0y}#BI(7NUZ%H}Yw%9oNg_?=f?AAKo9&T1Cns!#A!N-l`N@vOEe@`^a9ie+pe)?l{ zLrZx85;So(`_L#IaA<5xAsZC0Ba1cgiFfU`gYSiq|b1`v8UQ0hUXU}&)5dKaN; z;}eP4Q~U=>d?|OL(2vaqiaTC#?$qxqG|P^Qo`uvmk^p7P1&QKfQLLMQmD7D#;-hr} zSFQ-38JaYz@{XcmJ^pl#OWk7jk8P&;#-U|8zGUprj1t;&`IfmXBrsS2{8k3oEApi~ zJVDP5vocPk|w;nDjI;{*y7q!mOm3j?#JKE{4O5m=o#S@$0~>jz*AF6TvSo z+c67KvGjVt5D!t#F<+TM#G6g~`r{CGHUY7ky2EkQz7ntSg@P4HRPLb6bg66p|G`_#L&}dA5j?vfB6rFXqySf>F0(jSv{6e>6Eta zY}Pib5~`C|wn@`S z+(qE^v{daCqV8@>M*PsUg~XGYYD=U4MaquXLf)9)w-R*m-~?dT{v? zG^+vQ(>W#pMLi%?$F@>IyQw=1TPL*j4Y*b29|>l53U)q`(D#r$qPuT5?)1IcS7ZW} zg)GGd;!l5@jh&8a7#_-?r$P$TrQy#;^7#IEQcyqHy1QwJz0x^Q&3^# z?5$at=P1`#G=*sSn>_s&35hy>qzJ zOGZIOAnZRM&!MKllcL|7Cb2{s2Vw>r7~?%VR-roO$yy864B@B`p1dQLes!-Fh(_E+ zBV(Negx>;-4d1z38%k1Fij$WQ$YjH!?c#p?8O+2l9la*`UR?7PZnzcToQ!{UV_0o! z?p}1c*%$uIuD$YHsJo!xk#iIz4c+aWiFcJ(=AIrph{%+lYjefriv}mUidkd?!(~LW zOo;}xdfJk1(uE}d8y7)U3WbcSe=|jmg^8X418#}r^iu~mTS$npWVY{tC2Kj|9P^W( zAd60nvE~)@w9z4=4N_Cli0rdB6_o~Hr6T(=T49BG^KxXpc}|MCqQFsncSHfRkmfVN*6bmC9rkVY#K6A?s0c+}NU z>5VgC(~tw6zXf%n&YGU!X6rQ4W(7`S^K?A8T!bb<;vl9??5JfatGg54s8tLCmV#<2 zsW(q!$r4FBv6kpC2w*@5DIOd!kYHO1mo72st8E(IM=S!TS=&@$3R99kwhDB^G6yh4 zCO|;FAI8_E(~ArC+51Rtvm@jrDcu%SAT9}A75(D$y({T(itm>H1S{B2phe&g$|GI{ z;Qp06tY`(T1ehyFv|;n;1njwpc0E{*nT}A{TFef{a&vcDe3Dg$VpB6l>~T zoF~A)$o_@J7A8Q>!CmIct(>-T6)9$5{Qe9IoJk1_^mj`@ zGWH#lEL|;!t-GWlpM^}_;3!qR-6B=2)5ZZIBEwKy+g)}6kT07cn!Z*`0=_8?u(aMA#~=?q@#XWQUp0+18n|&M z>3f0L;knim(&Iq2IQG-ra*ElZMmH>YRLQWIwYZ>#4^AcK_Q-iZVMp5Xw8N&Q3ua(R1B;L`BLT^^p42%*qYxaJR%b#Va2z`eNT_GBBZ ziPWkQMGK4_of;cF7P@G zg^L1Az_pT+D2U&+Q_eoRA}r9cjS!qMeCBMBs=7}y?+pvIBR4SFoo!L_A04&Uwc=Ac z`Ks(qvwwA8gzc;~eMW51~;N^xXf z3R=SW8m=v3UE1>7i>v0X8}ZVR4nFSia5S`%9$x(}?SrZbP{Y!1e;+W{Gq)LRLf*do z6x(;%B%N}=D6Fgt7yeM~-HXJR=}*H1&sUHMldCZiELT%3F9sweI3n*FEg4X-cq$9& zM)(YHoq3k7n!Zj?y?~13bUMIcJXT1pHb!(d@7-ue;obaKZ;mXjT4q3>(ExOTP`yWI zX%Kn_Tp;@>x%f~LsywUHqLK@L=Id-GCpuv)_JQdaaxb0pxEF>?%UZxIPXO)Y<>?)hi<( z`aLmoV{?Y-s#sNm>pZPW#+Ga5_noH_ByFvJ?RRm%1`#$zV`d?uk%6ICfDW6xo>MN6 zs6d+t3iJ<^<%>Uriw@G!=>%f{A=?ff8s-gEU0yw| z1A?0>snvw*C0dEwcuW_=yYPJJ=nZBT_Z-Eo6j&T)o$5QJMW1-NX5T4)7Xs}3f>olh zI$5%BbS?+}^aL(d{a{|$_ZFd`dE#|NGf3-S*cSDj@A=9BEm$Dg25YeP2K-M+Y*WCi zzCsN*;1!7bKn&#jIZXozshXOCP20@S3#*4tBjf)i*^GL{P~brz45TOuv@D%91X;~+ z);ce6M1gCYC;+G!=$@)V`k$f+BZ1c++P-3f_)tLmG+m1KOFmui|G?AivDHhk-vAiO=|-@OP37}Y`lhkUC_Aa#3J z3WBUl|LA8&-0>b8sgHR2-+hdliFW`v(YAjB<9s4Lr%kSbdcr?P?DZXN5fB?JLZo62 zbe%LzoM%6@`+|c1plnlAnDO#@_`hAgxP!fDynUOKI~VmTcaB)AUAWTx*h@kL9YVTY zX8*NIkAbmL`OO)>?siq^y13JRn z|G7H+&*+Q7`G2?^3Vhkw+3f&;hrOhqZWovEui*VhTAF~sb#P!nubXMMd=)R?_z$E< z@xbnLp%aZY~lNQ@OdpJCh&IrAWH_D*V$OA^zvNfu_;B-M#qWg|Vjp13U$n zYX6Ud3*4rw;eS~s@E+X$fA_*b*Nu+mgWLHzeCsFm@L~7`2txe-l}i`#`w#Bz1UdNy zFuKe4w`H=?+rz8=BLhG4cw5aG{Q(Y_UGab-&Io!Mooy}f?LWy(?$o0{Kiv`E`u|`= z!y91AAbG#%&!6wE1$31R|87bVo($>=NcHaj=jg#hR<-Ww>iQ2>>-p~NL}0q42dU)~ z*S7BxeGOOLeLzL&hGlBi7sWv-feSA;lCD!>^{-xkc-1aClJOUC#nY_o=xBwVI`LYR zQZbE2f=*MQ?!Q;YwX

    lFOh;MO$0j00x+u67~zYA@@J3o-nj@J3}jjsSV|iiHuBI zeb`6@MUr3*QWt}lQY8cuC=3L7$t%&;CV;XE(x~N16906`-T6rvNFd@5^%FaH=kmPI zf^V^ph+YqbNV_sBNW2suS9c)B8?|wAkj5pLl0#mR1BK?rm@a!?PzM3wFBZ-9Xi7FG zdGn!i08S`94ZwRhc!WGVOFjG-Scr0gWMZ8kU-F3`@IUMc8yq`fG$Qei~|PIxJ@vE-^a4=*qi8g@82stejM5MB>afb^21LT zPyq)?OXF=_8QjJ~!!LcGNV4mW53P7{G%<~FAYgNafuUj-rWY@``<$DUu&Niu5v~Xi zv|Nq^PTWOt2(_L(e*77ReelP-6To9i{4+eqdT#qxDE<*b_Jh-&uC^e9ho>kCi5}Yc zB+~c%onNfOK?=Ljb9Ci@cORS_H16>=AUs^Ehz%Ghu=8o&%K3jR2jl+T$;pVx93Uly zhk~3)A#gL48`y;Fwy%MpnPm56mw@56y%CBQey0WS4>Dz-c!f9JMQk98d^LCf=l8pe zhsH1VQt+=99={k~BX`@MxDX$sdG_f7C~e`dEL?$#Q}3bGA@)PtALF7=O3N1uD>MiX zD20pYt_Pb!TH6V|__8CdE8-g4t{lEz>?Jik(p$oC{|3#`4GVHjn>XWvaPlmTP zMY@2{Jzc#x*36Mh^ZCNRMh9+Who2%B5#SSWus)Gr3pbM528(Wkd+7ZX5<)}E zV_Jl)R1w!DX6AdK;Q?3GU#SGAK}Ynd>ke5+5gjUp?SNKwd}Q}o100#l9Vjj$O^3*J zoN8%F$uHJJmw;wkl<@xj5y(}()dUO6iHW^IBnlV9&bJ@WbJ@&n5Fa^$)Rndm+=uY+ zGe{7w#{QxvP^iYEgN>U2S{$f>HcqC7ZkQ&BdO{Hz0AczqlBqTD3E+{#-vWQ1#bJ^Q z*f?EA_B+&|F#uXwy@Mg5N(LcxaXBdY)LlqWPHGraOyJZmIZwrI8s0Xz?br#j-dK#c z+}Cp0%tklntIu{B3Jr+hk*Gw4*1b7w%YbrQprCMPY*^NxBP1jQi*OJ0_FaO(I*hWV z>N4?n0FalOCIOhZ@(KQ#OfH#=uqKp0jvK8$1ucKr>0zNN!2`h<%RG9T0y%U!jn{1+ zEU!dXbu4dQ`p4EwJWf$rn0b9DULz(0)@2m1pwmD@0qIsj-itIaFTi#Ww^Aj+s5B#u zUO;#tjCl8h+#3+LQmwU%-2vZTafs;FahSJ+0|NsifpPK|P@EYAD5CYfAl%OAia_EJ zt)f%{Z39Nled#g-!yWl(DXv^jG#oD8U?*8M^I}1OPBRt+uO5&Iu-`Q55HnmHQ{=Ol z2}fZ1>6dVa5B*`o43c-$@gSWHd|}38U{)@OLoBpf`r{pbouf^TMo)og1!E|Gt_7Y+ z3v_3_3a`nW;V}EA6nO@aB|;y%+zXb*4kAYbSRAREc9lpL;BvZBlB#}7Vt@s(;U>FM z?^Jlx<3<}}kamETJ{sYM>wKzLujWJjji2s^>m9I=cYRv0TQOif0Z(zB27Pim+%QN? zoktrfu&5-9RQK3PQaa(c7jt2ZfKNb?qe!+M2umF}LK6#;H`@`R%xx4hJ`@7`36kRC zK4DJJ&-FcoRXIxHp0B)Zp+P^nt_Dv5MK=M5iv&Vc(B1{nNJs~e>{V(tOts5Ua`*;( z2`2H?$HP+iUv0l0A^|zD79y2fhYnW!okrw#0s^6NCtN6m(9|aeEcpJf)1%EmA(Xu9 zgBSzbMw|JZz&*H}3!}h^=gg(?ZJ@s-6fJzu;n)8eaG;Ojw|W+Ep&58?cvrCAP@q9A zj$fNm;l5I#d_abi+Q!W9T%7_FJknFpecmyryT%qw#GU^&&_oe$_*}9o=t;wE7qR_o zV_m%e4^TcbGB5rUxNd*o|LZ-L{tZw6d5%>7iKzaMfD>L3|DWyN2*OJM#2m_REd~B} zH>8!B+!fY_5edq67<4b#y1G?Zt&I7sL3&1v0d1eHS+< zH1aGS^zd~q;Zh_P4R8uSJ-IB*?~*hp!d4wVW+eY&MNwqF|zYxeYt-^y`z4 z*Wi`B8~4+VnG0vPUqqtgEts%rjsE3KZ89GeHi zLw;LXJ$#Y#yvpQM>LAbkjq3kjJ|iRg*ip+&!)a1@CwqnG%(@efpobeacLeAKjl?-Np zOTssZ<(~$)FuH6X{b57PKUxf6w$&q0eb!A`*qZIoW5O`dFHd?*JfZ=n)^%#82K6Q zoMd%Ed^deyd>D0p4u=Il=G_<_hfg@(N^C!i_{cBwkx6%)mBufDLMY}df#gd5hmFyb zQt5|NN6J<6ct{n}pxPnfgXf5X-wl_DwbUT7eD<`yz(P0Mt79>PfX){#>hc@l%)USE zd@bhyq(IH*`{{j~tJG}w!PBq9YmK7aKTxL~vFJ6O2D#7|BN@PawWJskK7ogQ6lFPnV$df+to||JviU6hYPtf% z6s}z6od`KTu{=Y|DMZk0=(`Y)Z%n?v?dp(RTetC2OAeRkRbf8gm=OAdhtRekvFbUT znXyW|d3PQr5yV-Z&}4*PUan2isYo-^c^SEGY8rewXBNi3iUBBRAaI2W{1^2546%V4 z$M2PEZNSTAo^X+}uJ%tTKTuBVH;>7n+jyYmWS*@+>td3qc*`sJk0!tL2HUVN5gv7#Q7@oL;ZkZ+r*rpB`QTmT-v zk15&(sR_tYHPF)*ZvZ~M-=|MHR^sg2t5e}YdQ$h2f%hLrghPC)6v}S}&gI^=ZB|sOq2eO%4j-V` zmk!BJBqJTXvKZ1hoI<^y!tvvJg}`X)C#lK%)${zOu)1Kc(2JZ|(W_~sG7nuA${tJ? zE;*RB#HIeN9&6+i`C6QZHh=H+s1sUK0F91^u*oM1rwAP8#-%1NiU)H$gRmz1M|we+ z&@ezT+M!eu{L>vNU%uogzI{Xa#mmywyRr7<t8&j( zNXz{u8N$J;mI1xRbqOB}*i!`We0x)&M=V+8%>5b?5PWof{NA=cfTC)CdqLsnb)7KZ z`BI86S}vps!@OrEzQdNCY}Z1^8U#eRgJ|$x$kcM(*IM+7mSzj?Z#AD;`NonfrKcg| zH|_iBdJ6JfhCyrmH>c|4o>~ps4u7f+z+Ubzc|yCf`n_O%=zGD|Y=Gna@n)I%H?X^4 zqWD8tISPC+(q9ne&p>}WAcM{Y!ek?TgBSaVnj-FyrAod$g!d!i5{HwUWM~r>l1elC z>mF;thIIv&$4GO&4x|)X-c|D&h`awO2qtN?K}&Dhvm*K(&OJnVlKjI>ciiSXOOgmf zc|Orobz9w|S{)x(C?uWcIX7tiz-gKuo%C05l;*VVrz z3up_rh+DQ*D_`d;`vvIQhdV6JNKa2~iVF|U=29SY@T@qGg&V$6J*U=f45{`GozQUi$gSNF7@~i}q3FgQwvlMFy>A6r6M)IvWWF#iW_zjPLRcyTOL- z+;!9Qz%~0;#dj0Uh zwY78elb%(7Bj&-A2(1W63vW6YxgwwH51g5qbJ86G+xg?6I%6xV6<~+;UCtLL^@Z)N)Oqg69ujL9c+w*-sY5y&73NY50 z8rZ(^WXhd8a%55?!+6MpgFuQ{A!EaG|_{Ss%I-W%0s_{Mv@f_HqG<7F9i;v#n;a63ArW?d6xh&{} znN&EBe{fn~aW~Ky?(o261g}%~oJ2wLcw+Ucv7f7;?3g! zTd2ETl~rKkc56~-x9!m?lFl`Kz(T$Dcdu;?)e2j3fU-!@V1rL`}av(_uz3uc;g(Ido8& z;eHUO>^jr@FSMI69KAtBS}pLuOj75c;f}x2YsW76N6xHLES+CKj(PBI*tOJZZWrWs z!Yy7J|0K)tS46?1V^R9znO>M>G4iuD*EP%0x!{m^Rfzg<;i@TJ13ey7;K5a zKDO-(C1~F9*i2l8O*3moE_KUoDco~IbEB^nLkCsFxIcl+}+1V2?)pRmYE?{U+PD)GqNk);yrMZx7eA7whTY=Hug+XrN(KY%F zeyn~zmAkQ><(kF%@;t?$hX3~hbLZde1JP8(v4LBd%gT~Aw>;Y2G-RK@nJ)Yy-r$)& zm&=Mvteh6vz912Od@3T6-Wcw+#dfZvF!x)IkQE)~LBtzXP7F-C^>G1pRmgx??#wks zi{Ixm?6v=JI+l1x#e^XIHX8Bs`50V9MTn4_hP~*4SjW_a#y#AS_d#7psl2RuwMBM4 z#WGP&B|peO6&t;mnPMwf6dOyxm7sNtp0;-W%EPX2X7?A3PK1x|Th8Bx77|?EPK;u$ z3^bH)LEu9_zxgHK-rFO-c8h3}>M6OH{HeUJPyWbUIT*`)O8m{aI3@|7&#o*vKbI1o zsycZZRLWJ4ymZP>((e&<{t<8J+9#Ls6x~8O;ahYsQqiez(iT+;@^6$FjiN zb(YUCS+Qd1TCYi25d2dg#+;V5v^{)ock0dF17~(L+Wo3jI493ft z6nzdHvN7c@?C2XyiCb$OCwp14RmlpAGpyg|{{0Z1N#m(BY@lyRb!IK(K-H$yDKqO1 zh4pjj_{P7wXjO8exN8pq7q{i|V$Nnxgfv^sb;l3bp=eUx9>WPc_U~WkmB-9x4LSr~ zx0XJjPYxSlDy`w8jW-^d308H>rH~pK{6_Xre%KoJ7sy+0@HoZ$oV+D(B|0Mb*6r=1 zDemf-d+6IxIeEr)&7}I`(we_gqeKFcKXdguW=dF+1`Gt9;^ObP#E{b&OOfjDgFCX% zcwUqHvr6cjbLeLm@xK1xw6O1U#J4Mc9GW)&zQk{bd&}oryJ_+>-EBWcu)W}qk*%d* z!A8mxjOY=vvqBbH9U5v2rvBk>9Q%qj0zvIQB3NYiaz?L;Gvf^SZ$)ZEud@OtE9E$a~!?63@%vVFvI zeBjO5gTIrcC&{@doG{<9^LdG?ZpDtz$S8XvK7H)hWB2s^y?r^`Fn5n9|pb)2^68PEW1>HWHE}Gzt@uC~g zq`O>al4X|HR(W!Gn@IK3_y!!X`iqnBwfKVY>Bn-K@<-8(Lx<&KFE(p2bWu$bA7#$&7%@K>+kD*&s&#Ar`MrMA|1x2 zb#Ya!FfC>|yPQZkh+nVxJ9&5n|ATCd;)0|g2G^eXtN+H|joX-luZGrry}AsRcOJVO zkz=qLI&n;PII`ZrZ2h)St@G9^8B0s`yX<1#3#A{C!)mcO=2~$=dqE6uAcE=f> zq6qxB$yQJjm?}!(GW6!85nZPetZWtB4v@Sx|VHr&)#k8(B z?vfI+>T>R>q9W|r=$nc$lz)O3=HmLyUGp|_=`%+#6lUw;j| z)yrK%eS$&Y(Q}$371tS-vStEj4J9ko;DRd~bxorZX>@BT4|N=Z6R}0bG|yLd+CE-h zcl&dkM!lziDB>@#ondZG-RYaAs$VDTfVtuq<12oT@CW8aeO(_{jCG(F!)upU-nU zd)d??A#|8F@I2${ri?7v^bLpJ52{Mj-T7A?mQ5E%`=>^tcW0p`h{K_Al#TJzichx> zbaQK4IhJnjbw$-T0_NUQy!}iL;|tYrvaM3S#C5d-7@?tD1?&C3hFO$f~6O5lt&Fc04teXq zL^6Vmn5h!D%wx1%T@wY=btY3Gsmc#*2{WP`9?s@+g&6wRYW3>`71+BqyCl!%F`YO; zMRHJW)v1wV0h?(oah-Z(stOa^E_B90w9JJ)J+Jr`(RtfRKCl|Q=1B74a~JeIU(zp! zwUS)-u;Z84I+jk@D%Tp_nZ87D=4!)fXOHP(Q}9lf22@FuW{3UVU%E~a6Ml<0cZ`mb z7@oZ%BgLa(w(_=bhp#7bFs&UVCa!x|Qk?ObS0`wOPw@N+s-#7tPqkcn-rh^>ed($i z9$AEv_e+Xls%aMzUbdsK@MK(dGRI-XE`#kCN;2^ArE318bsVK;^WT z7QAa+%*La|?94d(qyCe@uV0mx-W2Ufkc(rVoIOQO9+uiZ1ydHjIZ;v!_XTt3{*YRs zZLmzz>EFt9q(Hc&Wh(vtW{b;JEJ4)bUdTx9qMRM^meTtu-N%1arrupLin(GHIY~*l zNF-^^)){T7}ib#>_JpKyq@`75iNwdl)} zMhlkJs879~P*(4^x?lav5_6R_&f;emH6=A41YS_6TAXht5?@;!H?%z6?}Ghm)WR4> zCt#sqBk0^s^E1*q{`qEqzx3_3=41)e(y7>k`^vF2Pubc6k z70k`GYfFb;;3ng8kDg+eNb_!U^SZw3cq#)P_95;5H1cYSFp{-Xr`UOW1wuR@ELHU6i}ake?c`~NKeDwK zdRs-moifD0YrF=-L~j0JBtTyr{#F{N`uV7}|83HE>YY8Cy_X=kdyV=!#xb?Iqw$#f zMvWPcM%lSh+mv3@aM%1Cv%IZw?u@#U<_lQC-@eitHJduY`=n%(zSel4gEtnvy0PbGxcPjG?^D9p9556&et~2EWW?C1e(WUmt5U7ugJ&Q1J2vU1 zx8`w)es+HH=w;i6a#r>^qczHV_UeV3FwuPdlvM;?}ix zq4^C(5iKL`{s^+8nGdfZhZQBqMYWgc;dj>TPC}hmuh^GH+e&x?{Tw6P_VkuJd@hi- z*$8+&{EJfX6gAC-b+uVJyFa(PE1!nSXMDH#MIFWjGbubC)3>3#JL}JVyEK-jzxS5n z$M?=o#@7_igiq_4nwyK>ojKLV!4}0}x9}lit{r=st&o^td2-b-Zcog`~fg4>*`>`OQksTM@pFh&VS%OhgSHPwC)NwC>|al^nBA znSO;a<($d;D`!_CIeG<}FlEXm<6w#|-O$h}*KTF>BL>G#MUog4QEwpnbs&%<`a#H& z>+ne(`!Uw6?Cftj(PGQ%%kLcO1v8Tm^)q8`^$oehK=s{N7o?vq1uM>6E6jSK^-o&; z`3=|k@P!8Hn(if%N<-!YGEd~LKvURXrpf0VV)HuvztII7nDtN?V`jiA^_Z!-`gi2^jMFIJX#Myf zyE8Ouv8NANh+J)t*wTZ)m2aYOKv-*9jNbu*mZO%KQ^r1qX{zcu7N;4czZ65@Gcfq& zCipg)?27`-&8@WfZ57Ohz3ADdd|4sCra+9WQki5j_5oGYCz($*7K zlX?I4?E$^?^z;%#YuA#ojes|2OTS>%Q*x$=UhNHPx&&hmzVz&6!?==(;#+rIaUYM# z-)^i8VL7hpy+B_-a^3S)#^9iGF!@m-r*m2h-4*tq%WO9s9q+}j>@Auu>^YiVsm|Ws zIxutSX0GR}@}buPoaKMg;M}i?{vWR10xHVyiy9t_4v{Va15hNDE+s@!Qt1v+>68v> zB^3olLQ3h7W(WZZK|s1=Ktj4}i1$4B`+x7Z-dW3~3z?Z`?sM-s`|Q2XnVpSwrNOF| z_wV~$-mT=M)%Y#& z^x!f}U4ev-<4dqS)bxw+FfV?$Lblt4a9;dA?j`{eirT zSd~qknDtU}?qB1-@#Y1mQV!|ky#bD4Bs$X)y{t{g+&_)vmMe8oWl}Ohkhqec)oC8^ znH44T(Dm`Hs)~I@&25;wK6Sg#=u@c`L&D=M3wq))x@9%mAoo+Z{#FlarDQfYUUt+n z_6RNf7Gb#cn!2sMUvn2NJWh!ksm!3Nl8lYA=wI)An5#uH%#5)g0(dao2agH zrlsZ8avZaQI->jBQ!?IZl}`lhsPm^sIdck>x5fQ%@bl@ny-EJrhR~l?WquyuIY@A%F0iomJQAALmtn>Rkx!` z;c#nfbW*3t7dmr|3yl1XTG$<9$PIH0`~wLUgq^&0!C?lL_G2Z~SE;#$EAW@3B9`?W zhI`6|MMXs`@c(zo@%lm8Cu#9mzmfqU8LB8tzUquPj^W_jb1wFML`mP)=2J}w833G% zUAdBb8pW_T^Wl6WY++eS9nZ=jExm}O#K7%4*Z5t2GuQuq=n-fwxpIE zADJt_It}o9*LZm{b}3R*$K0(knA)hl&E;I1y1IrGW|{MJam}Ky4h=l6ig1TKkSC+B zp@EiIdICX>BDNeom5LPGB~01EX*ugp$O@l4V!zg(^CvZBqzXK8&|uGJaSaWFZu4pTdr(!?PhFcj$U(<&wpm@{BU#0 zRcxQEItKj)q?al~irM3CAW)0bx6xti@HJbZml=3yJRpHg$J-3iC!7Z0k{(w&$ zuyG0o7kK;w`2Uqg?Cf#-JL1R*7<^(g_E<;H`RM2APs2&-{ojIp<4slZ_qX1DUa#;Z zl@idX(ZA4~`mg*tIj&BR8O^ed!_-3#mlF(u0^(I}XL2dM1eo3u=9uw}Wo@UDwsNrj@EXR8O^ zZuKcEZ;tUUf!3@A6@9(&nI#q^3Y|Q#j=?_pNy!ZWEm1!2U*T(3O*Xu^NnvCH_58rv z5vz=x>}k}63mQgHww$a>aC*1OcvsS&%q6vA3h}F})x+C~Kc;)PUgV7T30NLP5vu>P zeOW1*<)G5)f=2eb=zcsja(4He2Zuv#q?cXm~EsRrKp`PA8 zpR26t(d=ozdls_=*P`y%;Zr?n^C;_x?&jIpQ=jJ!cBp1sB^)X6Uu8k`3Yz;MgmIXX zuLNvHM?7S#gCtlT)ub57(hQjL8zJXdsj;z zUT+Be!(WFDWElsx!M`8q1TEc;T4{sf-s;Qa422@2Lg`}9KFbYX6vOT6Td>Vn_Iqw& zle{1JmtW3X1K+K6I9I%CdfzIxaCt-Im#b2kUjW^+h2fG+-v8wWvccR@Tz`5J=TfIf ze5?Tx&W?qPtWok1X-@-0#`USL4`MNb^>6BWRxEY();t4&SJx1~_X(9VEM1P<{!F{5 z3J#%1nGDHzE#p+JApEl2+g3fKU~#GWTUGOsxZAu)8^f|u@@Go?;mIMCrPP>}5Fehj zH}O3!^Rl-{k4ZbO6j;21wGbpwuo#iG5#%&Bim_CBak!=oGH2Y)YM9BnR)?KSzlV8w z2wa;C%apgr!K(%>^5k|zYCq!zgI*jqBMN?W_4(HoEqhSHiE49 zCosSq1m44!rV;>G+2dg|E+cwM> zoOUPXtUk?_-W@dXt!ZdoTfFiVeYyp*fnGc&f#7qVHhQm$0%ODthZiY*t5Qp@M9Bwi zt7SaCcTe}~!%spOvaD8GTy^4|kkB*lAa+AdUhmDV&*f198Mk?wOTjuXrt49Flz*1>V)ue0Bl{Ra-5R$%}jroc8Y{bH_i-MrB|b$J@;K51STi8ce#-!NSj#A zkzRs=M!+b})(Z&J;P@KN0cUA6aB%rv8(1lxKE0%81%@|2=b2jma^F3ZyfdsSaeio2 zQ|)iyr7vRFMQ~aT+=!9IPckX4paQZt#JA@1p+41EW6jq_#x0WgGU?(z+Enw|0J~b&RJE1Iv}_miS&4f?50|*U zHIrWCir3?yWBKw|>+|~p623iT@WrU~^+BX^i4x1<)&&3VTCq=SP2W^VHywjeO08r~ z_e4?rx%i>1d3upl#;Sn1Fu^x3HmO$Jm4lwRU#dGb=3)`AEbJn^PKUO>Rr=JffOl>* zi@MKdED%b1o`aWIUIC%z&NetQ^Y#cA3RM&5jmsU|Eo%ShprcCNPHs+z&Yi{7?XaW^8t>hJg<%eJS?gb4bl=XEb76Gq<23HMM)(<0}HdNh= z2j2#00vnS)+K3qrY`XL!eA>K)Gs!5ucb9=jO!FE92^Lr41SQRv*Wb%CX&!b|yI&KR z?F}dHDzSOqnQ#7NnxANshPe{z;n&p`Qx2k@J0zWzVQ*UX^7(l04D0}>TvtOo6o6!8D>D`zLZKj!} zt1+LyI!x7U^p-mA?v9soO8BH#BuVSPq9Pr8>V`ccl}<}X=g3ZP^<^}7=f#}$rZKNp zEhr9ReULlPGb_%|x_hs{BlG5;(Hy3rieZrh{jB;n9UrXTg=TcUWrOj&2M+1Q&_Bfs z45LJ_?C>x@RimFLHzsSxTn4Jkh`T&bnG!lz$NW81Wj6)KD8{^PAA2)B?Z0X7Qsy*r zn4Ft2nouCx^@30^5g5VcTNpz>Sam72kN!W@*xAKq=U8D&ISB{VwA(q7r=NSTTg?4k zlr8m>-iEjBrh4HJ^~7I)y^dxw%7HM9Auw31Z~x1v@^!@La6;>{ytS;Gv~|K`y1G;C z*B0k!1(y1u*3EZ%^zG;ROkM4RgjY6>@g>S-<&!D#*I>^)vAYPo3aly5HIK@{znv=G zZ=%1+_E<~L^$oQAIBThkWi?g3rv2J(OUDmlEZm2>yD>nyT|y{|i&WS5D~*(Uff4}9 zmY&$P)E_En?dw=j_38?_0Qg0+zFEyDjpJ`4e`siRN=%dVjJBX04<#nK za7b_M^6Si*yBN44EBjxcl7Xy~)hODX@x`HaEw!5n#Th?JZ$N~iiO{1$8?I-{jTXEhtNAQgunZTK+`Xm znPX1P+>8-@L5ble_(h#2V%xZFtR}N0iv}pJ?4p?d_HA4d0U*S1Yt^jakgz)nZBD7c zMgp6*`|PRwS0;f7K%LQRJmth3@)?;H$~sw0QYBE-Nj=R2{*~$VgUA?f0)GB}-a2<0 zF|qyOgB`$Ceyx(nzAN7Bjh|m$qyeP5xzk#v&^TP&a}<&$iILY5h5}s*I2+IOIz1zY zr}>Z$%aaL#wYk1O($PO``JNL(aZ8H@>IY-zcG0IHHus>cj}ul{B@D!+KQRJTzP84x zH(B>fS`)`0D=>yXye_M2NtAr)vj2D*YW4e}e%bhgYX<_1)E7db)kE9FLFdF2~WY`p^ZapKS4o4bcQ&`bu! z$%}Yt;{KTEZ|DB2JAm@2g)+X6DS~VaiMWjcXLZEC5;d_4%-~bRyqlcm(D;qk;0E^M zu4C1c1ICK-*Ll;S5m6Xy7~IpvxWV-7%0OTvg>)QGftaFI8E(PNb!@Ip*%a5ZAo3Vg zxyfF)vo7)&%#1(O#4a3#R97{u2x|~&SKW+J{TI)nR~-62&y!T5rfQpV`MN{5`#jsH zS}_ye$LY064kZQ-hg;t{7hB51WXaK&m9NRR@6h$|cLi;KqT5x_i-6Z$Wm@!4I*iv; zJJCmxLKFQg$>)*D4r&P9q4d!^AM2M=N+z0%PWdEf8uKDXRavcn0=6M)lTxJ-2Uob_ zevUbCYeKFSD4<1L_Id^L?Up`;TLb!VVulj5&Z6G79EbjBzgk&^148rWLO(U&r1sPn zb+<uT&cYh)Ga?j&(vF##V$4 zid1r`Xb9N0*xenlzoEVFl`h^?_qMUFOJQ%vY}ey+ zlaE+j5+lHe-zmHvGQ_w8UnA3v*uF>Yb>?KD3zB1^_pr7571C!&lSCQUdO2TRdNh1B}YO;jN^>=h1bwfl0woQLlN_5u|5Y z$aJ?kou;NpkX}4!FbgW-kA{~}QToNN^|iKzf5o`7>#WKrVEk0RIw1sj=))t=rPG-< zbC$V3&Y_fWhIdS#npJyhlmgtiB8~~Ays1k=!fO5_R&X5t?0voh8LC)GaNbeKw(-a| zpSSd);%;~SD6>V~ZT_EFn>(JZ>&5L85{e)Gu+^{L*&d%~lDJosT?}?z3XL9Vr~2y1 zJekwKNTQAYM1F~A-}t00={Or!5uNe&#MA(x$TZ}>*NjDO`;#rhd7mM*a~LQt{quhV zF3~f4W&<1IZIn6-iyb0IU1ql}YOBu#UT+ZjSg8H$wHrs9S765|XIa<*t0cPPfk>dJ z?@*PwnDA zxcoIGE~pbY|HSl3L!Gd@Hi{7~voNL2aCk0A=?~P!nEV(X3RnxH7{cao$kmxV*1HSX z6RV7qRHw=tgwR+ORR2fvNL_XJgKQ?V0$VWv#fv%^k9JDFUd5L|0m4OOTv1DEovk#; zRj+3TE3RqX^2mjhv1xYz*^O}m1_laG>VR0{`t{t056?@uKW|tF zKO#t{^i+H1{?*gxYm^>l)N_2a( zWObMU2hH6pdpaJZE@gwxgyY3Q5;qu9-(fIDd@|==NIy0d6cc+G$eczr=9l^+z>gRf zCX(TDQ=?@0?auD~bkKy znWW9GvZCvrS|=7tcyfJE9O`aIoHFabDIkQmOc2m3U^y0hsZvo<`y3R3Ay%(LKU}ID zN3Y!M+yn`|Zh>WI%*4(;s;=oWL~Vw>>)ojYyiON~+!#sX3#$k;Ha}PkbYxXO8Lmi- z?_p%(dLunpGkQ>YvRKJxWh!5#Yp0`0(*naH@0eW`wJ0ljmKU!Gn`zhf6$9{^W`1=S z@XzlBMccU7PP>hy7kZjfo7c1_N=EGN+1>Nn(UTJl_zx_p+#r%=)I;CnN$)L2cumym z%)Crpy+MM5aK9fAMnV7`!x~XdT@ScvU?Zm!@I&H`X)$$4mL) zjv#J=YzXdR|cnooxB|h6u`>M+GHc3(vNAeJP!0+r76oNGLAyqd1`F zzVEAQKH$X|9GYG+eXlHyI4d5n^Yt)iD?ZXh=>MT$(d^Qz5fPH7P(VL(s$W>Vo)Ehw z1k(A1z(*_dO(5oL-*wSU?~Swscoua&_vP9wz#R2b6NY)KxfzAEvTV>|*g5iwU6~+~ zKqZax`TV^z`8k*jlD^D{uqx67VXNNNdGzpmINUMWsDyZd78wSmVtBIX$GxYaJ8w#` z?d~TD<=c4iLJsv^#)<2-JJb99rxFx3VO01<>!sxAAiY1MSA!Y*8KVwN?+JSlf@jXo zCb4gr^w7Bu*XKXGm53j`lL*FW0*fP5JancBg_Oo;p5EG3$dN|&%xF+25JxUK1WF@&B51#KdiIY*Xi9h z^O`zk1gCn&-?5&;d)WIZS~o}Tr&{;?5BPvC)4bGWTYt_mBel26)5m$;7(u6vev_Z0 z%V3sxusUM;beCj#_oW7@AY1kaj4!SC?@6?(RhHGPVDS;!?(M2}=>LE{NABJeyuUPG2uiz5$-)qSxAA9=e`FTqA`ogFi5tb>{Y4t_Lf&n{0?SEIC7d>-v zvB4Sjb}m}JQC~YyhFgXvN^NcoKGnu4nH7zlZ9q_(fBUxnT3u&nV1gNmZwPM&=lB`; zQ?4PU{%Y_?&GH$&C|f?Ki@k!5%A>!v;&KP;J_&IS_eJKdb1yIFO>qM8Ttw!zPe!jWE86okYM_;(o zcQxoQ7KH*GkYV)Q3u!LrJK*R51+orU#>*n+6)B&P1dmt<; z6cOM#Vc|ZeAn}gBOmG%X?H%=PvecEamyjA6sup_cfD1aBwDgUj;59)*D0<#ep8*i+ z%~!Gv72?sZd%trWru~MmRO)P@bJcO| z8p8P9*y)T`)2UvuU#cLvXwB(7YT%+}Q84hhR!LjTC2?=gw8dycP?&FnAC5#-!;oAL z6$2e&{a}XcL@`6u374Q+Y5{f%sl}KU4*l`aV?b`9SiaX9Rls zg{C{I{7>edazXJMt5-uDK{)=T7x(E{`70$t7Ckv7PUItTTTNZR_BF)2KRi%q zC0$x+=t|5!QV;=kqv5%@TW}L~=ddso{)F@f4jVdq9kD>(wNtwbWJV*YS2H_){vwnQ+d+qX*GVPT5=}b45G7*fmGk)Ba z=)7_4roEI+IaLjRx_F;XZerit5m-CmIS#tqI%ivsF6@$+S^HGr1Vu}U#zqmMJ}`>c zcQz%BzPX=a7U`7ZA z!DPKv;7$YmUTF`)muG^Cf!sjy97e~b{;UoCrz2Bi71%y17E^OQKU~tg>qAa1W@})) z{2w=n0sCj6cjk)Ktc)kB;0L@ z{xckAWkvE#H&xuH^9v8V(qIl1Fi-@7{bsp24t!_75=$ooo)`87e_AINoMk{PU$4{v zI1e8Xq?}ryE87+(FL|vmXJ|zb<^k3$I7sV%c)BTY3)Jze9#j-U+AiX(?6;94Yqa+u z?<4U}W^b_zs0OQJ5B}nR+-#3}Nz6gFvgeYRLds93661iW^eTFX1uFxxwwF*G{gol$ zk&?ZG5tb8du>MP%XgnS3{3c=Td6xB8a1`6dr8^+|zApW>`uwUn>4>)7xb}u9JQaB) z&t#x;3f*ZxKJGG0IrkI&V509esQ`JgM$V&8!#IJ4D&Lq0Q0W^ zzXLc~NARl2NgbnXZFyjf+W(z{v8g~yy#!Xx|2gX6#nT|7$Skyg>iImP={t}fpa9tx zY`$gpAVS&XDpKUgvA(|~{G1Zj34b9t{FNU(H-BwPS?`hGLDV_c-cW?eEovw`L~6&5 zW{R133W>4KpahV`IF38cT$JbLKEZOInRw5lqebZQR?rGITPbV`d|mg@iR`gtsUG(1?@_#FH z$y{>YNkliN6SK&zeek7ke22!a^j%8vh2qO3*nXFixzi|5@NiRskr9<%xt(=lF4g;C z)1UXrV+I_gh%Hn?(BA^MLKbjSZO*aSI37YY-d&6u1wQC?U(|cgA6FRSpwh79F|xEh zUt%p!rFJ7%H6t?N%`0$#B-`f0g22nj4&QXa>@|R$p~>|yOfY}$|D z+TwVL!3n_Q&{#yJm6)VzTTiBR;89h;$6Nd7l+3&bsI{lX(%KAE&eZPwoVfD05-cis zswD?@gsbnQSeHhc!u|{BDJqA#^`(;Y^|h!~AaQ_P@T$@upZf+H5Bwuyecr8S!P5J8 z84#XLA^c@#^#Vi$+!kvF*8~g93^tS~y^nupJy> zi29Yr|!4cd|THx@|s$;mIeWS(Z!qmKf7=oX9e?t0Ael%ryDSjq>x`4oWOHE z(Oo5q_@{pz8oa)a8x#~!*Vao(&K6p@zteqQUBYF~B2GAqX=cpE-#MiTG-&Pio(wXu zh?A*gsbpE?9DOO40a3Z8;HLoGGfuY^%a!U5u-~2qH`D2|o5?@nB#H2IxZiuXv|te)O-(QZHa83a|GJ|ebB$94jxKM=9rPLxcYu; zt{Vu5gS`HQwH4zW9{Z?5qE-4E4HFVDKjaa~c-^bFqC50~r1o_;$#7Tdj( zzl9I^u4)tuIVy&LKA!!dvj}S;!x&b7x-KvYzIAhHYR=?$?&!^D=TXdTJ3>JsY>)!_ zjsym}?^r9bl@_~pP*CCL)i*{IR$Tn}m;fAAK;yLS)6b@DBEII>%1TJKz^&^z(HJ9V zVO|6r)8UEDO%S8S4*%Rhdo5&UT137nXM!Sth0VPt#W@YWK?`jeF{WRN-`mR%l~ed= z(%f&k%e)ZVWn}N(#>|-af79!oV^Mv4^o2|$J2umR8$WP+MTDil)-%rbueSL0JV9>V zr5oMexIM1VinVpoFNnFgxNM!Kz5iP@5xxightN!MKJ`}3!zx7ph-Yrc;|Muug`h{T zQpxG>7f5ly&%T4PU+6H3%B1p_9CPuO2n&PdA`oMfP5n9^%vzm~PWKY4(|&2MH9BSF z2R3UFFt5axV}3)4G4vL)7-&9_qU+!ko80~LIE3;mu(;>-C21%o=DwU5x{7J1Z1buB zGa5k!0VwmENXM9GBF6adRiI_-^pbA(m^>n*;c2;Yr2?8*P@5^QKM+RJ^_@dyX;eP5 z^Hx2s#s%N}=dx4^Ks=!r|3*h7#G?mkU|9L2RXOKuR;zbcwUz)1A>m}mS>UY z?Y#0ZWs_$s+z~kiBfd-D0XD2Bh?`u9^!h#D5(lA+va+EK%|Z zYy6a}KLv~n(i(Id|9IdVFKnIENlBwH>v>xWxmU6mSI*Yd>cO{VzxaPVPa+kg;&y+rth#+uu>3Q31O(JO_oEq;W?$&&Qh2|%%$1B1RN5dg7H`(s9PEj?OCVM4j z?5YZ;3PCTUj>wizR)2~YN~aM=OhnWB%t-@4n4E)sfvW`pfELUK)Q=NLI$r_KX_D%O zEU12~A0bwQjELuk5I_OC#8zOEG$|nlD`J4_uK?W+Dm|X(8cB3wR#=Z(RqJubX>y+hKe|@Zy1d) z6i2TJKJK}dL$FHW{c+I87ApKUaZ?$qc9v5{YEEG(I}i^w-S7b$x9LZOgb)UiUb%D$ z4g3UlHL6@5g@V%({;0RY?+lK(Rw-XpVq^qDnY;Iu@uyY1SlceeK zW6dM0(NY_Odz@m;&#Z(sG&IV}K9Kg_vi!trA-&Mo*O&aIaeXT)&GQz~@Nie_pu1(R zK}|S$({UIK3^-WXC!QOt5(OvI8R;_T*@Ot$+EHYaL`1P_6F-Z7uQ6eOOY-`3F&ud$ zfch)YJY8oV0|l&30O(=-UYFi^?E`Ss=Zia0|09YWLk@A2I-1X-{DT6Fp$7#VmyN*K zLX1h4Pm{Vk2DlFlt;O^ucxtZFzD>+PPAg!G!1ln7=YES#1Fz2-+?$}q zBWhJIhyIVc-SRr=w=+YA?UE8lGI+S4m9N!pgdoi_(8(5%@8ouu^&%0$`Eb9rW^?eW zOynvDhs(|!alZjL1FSYs{r)Q&tnZBh>Ohc5g6EFq@Cj9Iz*Gev&J`+3W9}(S_MTfJp95{HpDRt9KIhFR;7a`#r8!H(!bZrD}DR zhZcb;^PfZ>kN*Dnn0lTQcYoX7*Ifx8r{X?y)Oy)J*iTmM?CB=V@=MNlrL}msTt3Ks zWa{@gm=*rP1rl7d*D5NIg_O*Uo;mHW}fey>zCc zHivCRL%C@x(kDMH-0oX8e3r*m1sQOHWl!Zs7g<~J_2np7FGvG5U}Xq-#1u?Zc&s#m zh5T$hhVPUMLW0wCK42O5|31ZXwlhhNuiGq?j=NG{z8jkFUY;tc zK=6sSwJu0l4wQ8+7AZ*{Qaoo$)gJlszn#dxHGulw#N_QS?2v#%rXpv?HDd8#OHEqy z1=zupZ`=Lz9Gew)t=3BKtB63Rm&&L3E!j*B_W#(}1sacz&&MM+7Ue;kaQXqHUvsMg zLkqy|A0R>?_X2=EXa;06y#9WtJcE0IQULwk_6W-)vmo)tSrrfW>C!mIjVZtmt*tQx zJGnsUGoB7+MEpE+;)CV2Xf_FT98k_^9TQ|Dzk({}qV4^~&)I{X2>m@J|QH zzX9tImym6G&-}VHRv2R)aJD9WSL)yvl$y+t6+IeJ`7*RgwFWS^em5@nv&kr;!ZKE%IP_j@6Z+76hygs{TL4|@z)&lc6PHqZw(!*FO_y zJC6GLf{f0KV#>X5 zYd#Li!_8r-EaKxm1zQ^gQL7g1-ia^2`c+$VTsPWSIR!2HZv2YnZ(W|g8%ooeS|Z?7 zKEpr{ZC>m=h7@C7Q@yO>!BJq<0EeMvZVR1&5rcq@!X3~+!FXj0U#f+GwaOq#!$;rC zGO$m>kyeHM83lgS;Tg3+vMj_6WnHU^WE*h$>Qvrsmel%eFpg(a*u8~);x!JfA4930 zbkI<0Ch2VP(nJ%QPF=)z6VPq#H=lhDS{>3Yo`utD7DGQTEc4i@g7=>;#?!N*Us*Ki zgpO8&CZQMYmPFnYUOJLF{y6@;vy}W2GO)4i?jq^ zw(mlF)AcX>e{z+X=L<6MJYBK}GhKS@){OXpsv`_E&jo8LSR={7Miw0An%p@~+4$a; z{Dgh^KJxYHWgtw$0z;uuW=uCe7&0D$X8|qw2YrNas7WK)k^8-X6Sn#@M@fKZ0f$~k zgXhw7bIGp%$yJocK^0;%Ak|)eOA};f<<#-{j=~VzlzdYoFb=oei*m+u4XE{X2(0LB zYM>E`wO^HmH0-%LJ?hW~;~-1zPw*W_db#Rr=MzZrh|}F8e?z)6n0F>q=n>X&(Vf`7 zx_{`_N|=b@3O;clX{Ou{c3MCb1`7X~CW1x2Al5EjHAJ4LNwxL;mPj6lb~HlP@6O+T@NN$kyQ=?_VuS%N2kd6$$10ZNp%66af5q ziO=|3Z)8DDUPS&9$ba>iJO^dKiEz4MA4mc9ZmBz5WWL3D2U?iIEXJXO3J3A1rU;ie zL-7C*d7vZv&|RVKcg?X^y^+<6)C6(2qPhJW0q(OMpZwCB@v?UGfhGgl=(5Vaielf{ z9=Gh|%*zuRX z2>n|>5kEtpbRBRQk82iebqF_|-+#Yvtdp*(lULu+;zsnsri$5qiz=bJQB!nr zFRgfBmyJ5m9~6^;cuDC7%9FJW48Ndr-4ki}b5bL&M;cI(qvhhVDOXeNsqDI>0U_{0 z-7OSC$b);KK~D5h>@6`N7leTbv{GOyZdQKlL17A5UG#<=&hn_ z@BXH3`hz%r|Msrs>eTjc>#$mbE*ZQNVSXS;tgUo>AD#f|} zPtp#m!!%+pJa7AR9UO1|M|FHdQ~DENz+O91EjH66q}m~RPT5^-0jIKM=8 zKQnYA!V0bZ9g82RxJC;wOykqL%Rc$ZpvNU6^eWNzP zyajL#^w)$UWh$D8lu#CY9I0uRwV=+0U?w23lfiW%hYj*9Tqo?URms`O(}8}XpF0%C zthYswAl%cxz);2U?^MJq0y)gkED{PpopX%MyaPBk?d-$AE%+Hxm^#DtrMadI4s3bL z!69M>Q;%syyRgC7S|_@RUr#vaouXHi4ph}5#LJ!QfE$Ei7F_DuV|{uG*<|lO+8h}z zO3yAb)xK)(WS}~~O`nvpmMVrW400C*4>NkA7qHC;CP~9Z;SNH=uy91y&vUr5nSG zM!|mZ)J`8>+}slgx`DE7h5sp+D4^+5J8$Q(n8vMj;z~7_$){b&Tn17dSyN~a034&fj>|afDekb{4r$(qqiumGk{m|ge31-9;uLwuxjJo+Pcb5V4oLF^?|(TFLe%6&_!S=t2p zu2GMB`PoH|tCTU@(7H?5y6ZwpK4dtcX!(mgy-fR+)wK-NW=%%uPDVMP2r|GU@QLm` zHa&~9z}F<93@82N{QD74s(#*R{<)QGSA^kYtyu}Jsks_533rkfGC{M12gL)b2E;4& z^95C7ht+m9bon;|%QG=0{>wPc@BtCl=D9(-Nk;jgy_&bTN^gBJZcsCjA19AtS5kYa z>2uwa_glQ8>r2(trM?T=obtT!5Up%ht-4D2ewv^TN?Mt7Es<~7NXR&_4<;~XCsVGI7=>W9BCX)poL**(4 z963mz%i)*j&%o|41U;$WZhSsI z9(u4i-D+$XzL+Ac2_#OLmwNTLrrd4dv_q2cCujnjDaf`B^OW4$y50l+cDs8mlu1%~ z_*EWGZw1dIdu8)vDUJl^-~a53wC>8hQzmRtEpG-w&mB|cl+FA~Sc4?%tA4(z(n zU*F@3s5LCLCMa9~t?;{cF=t;&r|nkj!a(-hHo~huK@`AE@Ow`aKU|A3!a@^vVh~B` zMejJtYEh_M>T=_T!F$1ST{-ZpbcK%I-7!61u-7A=b0&tWFpvU%gf5XhIgTFZPVSol zRkp58EuF&tZ`U*xsgffO(*K1DcC$3U*$?Nd=Li~)q`|}m7=}6kH~0qcMXOV{vl0bi zP{*gzFL*29_@}bE2}udc9T)jyeZH| zp(~LHF?z$DVVR?+5E@$an}(DO9sPHJv0^w9!JT5lN8A~3a_WKof~DoxYFr(hd*g+W z*x@sOE`eSF5`P^GuR%CSsC^##q9MePt^cM0US0|Y*ZfJm;bT8{K9mr@5uRtZg1t7P zyVluGVMM>jMtEMpivcW2PfPf6ldLXYJZ!so8CoBck8PTv(nMNgXNRX?j}WRk@i$6d z@I_ipti~&x_2q_gN$mjICoj3VYzn$BLWvHIOXv4jHKDbX2FzYJ*OjQ44T~gU9tiZ_ z%Kh?EY8&OAAA}p**#M0$ za7+;I?f_IklKx`GE_6MzoJWfHD-4zcUX7BHHjd4<5Ig2I;X}jG0&VgVBT6oe%~bhyi*L9qJ|pxHfG5rB}sm+ zR4@E97dX)n%RMe~9)vXobTU-bHd}L?1SCFecxYF@rEL|G(-G}NTQaQ6xi$(g^A?&j zb3XR&u!>%{izxA|&I3=T5+2J(Ik{F{>*h{YpGsn4<>s%{(s0x2huGH^b>+;F3E% zA<1E01iuVSm;w0#I@?T&Z(jz=ei2vCRy+Bs<7JPMeC&Hd7wQ79P1sNmDXa35W^xLXy%zg`8qLu}DF$Zjy&MGSPY zEKg-0iQg5F<>SQk3@vC5Kr22wIK^;b&bhzCiMA>1w?Ey8mXL`WpvK(Neg{O-Bn*9eCDlkg{1$rY-;|72d)Zn5u0!nf zle`F;hb&|~H6+Nnj0L%kY8YGFPHJVo@-n*~Cgh}bu7Jdw|950A9ok-PCqnPK96P+H za;1I=4x*(m6)TXTkxo{O=cJqyGE(L>ZFCK}RTW90U{+E^Qq8RWTpZQ?kKe~;-M;5M z#SU;^(hsa}RDCxhq@1PK{m- zmz#Ni*8%VdDtdZn1O+t^dIV?4987CuaJ zOFPqwmLhREc!(aXBY;D010rDY0pjFGw^BQwsCT_>eo{y4cNcMl5`;3@)MN8ydY%<% zo>EVSAhs;F9*m^YWkLTLQlsp^h$M>F6*T*tVZ%lYw~7U%lKJU!`mRvWfviS${Ill( zI=?_nWPPYpbStUTya5K=VQQ$ksl$7LSDY$c=$Ab#MGhwT+wxs4UO)4uZ`Xh^V$T$F zdzY3*Hh2L9V}P}U{4GWUlh}oh4@srACgHw<$L$VK4XlR>Grl=+6*4f!3#R$OIDw=k zxJ(f4LXU^PwWkskY3p%lM;MLEfu}rrw37cEydA&xEjFwEpyZS33amj&$S?#u#504H zkpK>FZG`ZZ!u}Y2=8@99N+^D>Zvv>H-ikl7zk$xKtTP>lT3Z&o3`D)@4+^j#1ij9B zFXe7B%YAoX*NzIA^6l5m7HLYT;kr6+>`1T67fA^X zXqp{a{Rm$e(+-MCR}&IrUQFAh@MS#S-6j9g&?r=KpQQAK-_iYdxqDIqG1!V zQnLptXYhlVz3sh5DKW5@WQ$q&M+ZZWxbf@hQie*stZsb>=>w%|sx>4W3y1@ct=Md@rbV+oAF|eTL#TLY z?GaO_2eSAP?QboFK!?qL5=Q55Det&jo%5^r5#a);U0k>!_$(&%ZL|&;b8{_e@gTg~ zVYr2b7#j;{&rukEv$khoML{JX_r%CeM{T=GbN|sNOT^TYUImj^N(R}&qidNdl?TBJL$M+3&yIwX;Df$+$F+fp;e&gEM{wSW#1GUj(I2KsSP;3#Mq zUJ$@!qIbaTk(|MQ0TyX2Iq#HLWpzeqTNeOAJE_%6{*_Xg2|YnwU%e_Ev)l z&pi@)o}4bV1|uVR1cdGtRkA!V{hcqr4mCn>=!Z}!9i(`+Kmm_s_zLX7lPxir z`*n%k_7*VGC7LiQFVP{ev$M^U4H*psucoW#KB#xA^@6lJXoTz{D=>VG9PT{T4J|H@ zJj5geSvWYlW8^sP+4%T)rQ)u{KQ_W2*Y7k~A><#kpp(Gq9vrSwZ$1L#5Tatj?7?JU z{UIvAz)jn)43xVElpCioAsRXo%-K2pm@UuZ-6cuClB49VNtstvd2P0hqsueW;4SIV z(H#dz^oKbM36YQX0|)g3ANS~R5PI1B=z8o*!0guaK9D!Q+IkDnUho*0HcW)tk9Tq> zxYPU;TsR^qfu9xeN@D7sKz}h}0Yq>`2<6lN1_*#E`fBS_X_9=JFA%^WutoxPg_ObP zYz3=!1d|93VFx!4)li=pFfepn5~1QdQpxGa7#n8jq5%$2S(Ej}k+G(684Z|L0wCW7 zpKfiWmTM%rxUcPEa4DPi{xmDghXw?xgg_7AXPS59J1Q^N@oX z(H}Pvu`z6R-UIqH4CyTpD#SxUq&ly!L(kV52Umh{m_R>TZlkT5^^#DiEJ111=aTLA z6)nMjQNhQd9N425hZPOeVK%>Lpo_+&+v^Y-Plb~BAKkMNLvzq(cQy_-gHtzS8D zeFrp}`*^{(?Q-Z-y#5KZ9S~;GX*AG@Hb`{Zo;s>h5!z$!*JOjz`2*iM^H(g#M9*)S zX<}9rgPhE2EfhqC>#^2xS5H5zuf+kfAF=FI3dYp$!|aZ_g_x4FOV;QSFy|ex>7R_H02PRe6Iyx`_HX>ljf_*b^D_a=s z(FTo}G34ZacY0!c8c~Dbk_)MI0mdVKDFH8DWC#`Ed*EMKzKg&Rh(A&=$Bfc91f0G8 zpp!y7HT#GzOozaO-HaIj0WSnR3FT%brv4uI^$pl_k?t>4ANOaqOgd!SKu5wgT^f4d zGGx>8!NlU*lA965HO`Iog|NHN1C%@q-DE)Y2Xn24ala+5xk@A`2|0>pj(0+m`iIRW zWa`z?WWO$Z&;OP4kgWtvyI2$IX_`!(Dd{2 zCVD-D9ED^>%EpGDfVUhT+*uOj$rc{T@>A7IcoU?W-~DR9mVx31Rb3qp+}dpGuW3NKX;D;LS(@5UMb1I4(u(iTk09nJa?rj$ z$p>Ps>`D$LvAqU+HqMi+3TjnY{S982z zNW|D|%!YR(o3b|heyfbelXnR^{7W-aKn}i#1ci%BLr?^VU~}U0kbp}7HYBtNP27cK z3Yvt_KNNe{TLYv+?TgN|aL}R8NJ^0Dup&sl9vD7%|BRu}H2ONUikbH{X;RP>@h`!% ziCn8jx+@s`pjB-7!O4+%z)rWx5R9)cdp|-AQogBcZ}PjQ+k}cF;*Ve{dTllUnYFm~ zC_puY8eseDSu&Ok>_5a}qTOcs_BqOQmGv~q5r)1;%-=qRzgEk+|4>e*p9T_UFce`Z z87aEa7f-DSsG3BLTA{-DH#f!d^gL?`XBx8+w>0%$RKc%td)HHs@zV(y94z*A^oJ1S zE81DOJIGo+{1ax`cJ8Za^dE6U_fV}D`Z3=u2_D_Q z$V_PWJ5S~=T7$Kd4|oA>4iiTz6BFRGVUUX{%M`GiSdIj)8pyg;Ip?kOi_gI4sOX#d zF*)WM+B(v;$M4xD5*s}U>}KH=rx(c$-;?Q#ye4JC?%1esUALbbxDWrmD#(;y0s%Lo>kEX({l!*O=$?soepI9ykGli; z?aCH)djA2}LlFG}s6hyJkYOT13k|r4!8AhOjJAto&+E!$D+3w35_BA)RQB_>i3kGn z{@k0m_Eb7tz!fY8D;-2K+QH;!qkuwvMVZQ27rKsPPF1bOn&fYQ#E$ZU!46x=f^4*fNbOu5WWLtRC521d6R8Aq-LZ<$Y zk^O40Z0zGQAW#S{Z@@}SCPN*PH_)d1Z5^p95b2&{-kUSo@9CLenP28Xy)`5{my9i2 zeA{;A_0@FM=^34qKKD%mud|3oCCSu4s!nXa;+4<{`zpA%A`nZFJ#u-%!EU!*C$|uI zOdzvR3SAm&Q=f6&g$lV$F*Qq#EV@%&FV&V3X@AL5nIlHb^PcQ?M=p;-7Z zG_FJ+w#I{P8PWg)I_@IjPXEQd*uc!CupY!ie2f-*o7Bo@(3n;_x@Q zS0Hma=#Ij7IGuT*G*-Ue!gl*rEq)6-Ss+(vOih#qu)QrJ!qKaPs^B%=YwLvjUleZe zg}<6owGxT?i?h&DQWrPoo0?gW?oxDf8aQIilt#H^Fd<$zpAIRZ?N+EmUOE8k?qtR* z9-i)ahe7j$1#5k)S^Ua7L%(>| z9{J#lE8lp;BNtj6SUNLvliy~5uop4a10ys*(i$62Fq}^Hzht2Ke#tp9^)5*XynkO^ zgfg&euh~+&|4(HD#i3I`eFh6%GH{y8-uX2Ou0+0&Rk;Zznc?0>ZbKJpz=41kkRrOT zqs4iIdI;VZ2vsP-0g9iu3vurdKFIqD(s-!gh{7I##<1`=xvJ_na$2Iv&NvW;sGNBA zpx&YsitsaWO{kK}lwKfiJb#P^!x5~W#3I|!=GmU0x$#60fQRwM#}y#P4#w6GL%cxp z1z|Bg3$AD|G0*(-!#4Z42C=j~RpOCn52m7K=!>74T}*Np0oda+XbRChLrmm>zV%z?T_O1Iy-9g&x_y}t=0#Xp>iR2toCxmH%J z4yOHBKm~#<>@9GN&DJ(39lw{gTn#munMUfWaO+xH27FBPGEl(M5TZ}4ujBWH%A&1+ z2DU)|kM@OYe*M`HhN6U3AlJ^WN`SEmwfDik+Em6A%r2nGgz&Kb_OL1cwo2XVT=s#c zRdW^x5k$cQ{!(}>5%4*)(4f5LUuyyyQV=JFpHrfMXnGl%G$obl_%?EHHw9wskR<)>f{a2(#>m7Zd0?I*rW2C&k9(yk zC(ICZE}HPpadneVjGHHM!GE_jC6N>Ao9iT$gHS8X^UnI5RTP$(!Z{@Y#xr=`Ebrl$m^`Tv-ZRX{YC%YqS@i@=Y<>dGIHL zbRtrmt9)Z@v&JJvQM{*NGx0RE!*n#$lj9>|}sCqgp! z3kVLsYS_=BfSywcW~avH7x zRY6nF*&WkWh2~; z;r5wPXexlr!gQ%S=edcQI~jJ+y!$qj!hF=ivfG#KJjao}3Sk2OyUTK!&A&S?L*rQY z+5;|qC#!rbL?cDcGH+H&_I(wHa&PO+==c6~Ydh=u*cY+rkyCI*K&>PL1Qd~N2DC#d zWO2~b0w|XV3=Bx-K#E;>qN_(E6kbT(KDG$-`9L5;wF=(Rb2NMH3u_Er!5zzGDKLZd zhF%V8qspJAsSZ05^F$p;S5O#0Thn?hQidS29`<0QNOnW&$bkSr)kl!D4DC~2qGVXm zgykkK16MDjjX+AhkQ}L0Fc-Q_9p_H+7fLlgpY8 z+VO^4ThJ{QJzNJoLd&yny{q7aJq&~_M5a*Ur9{J*TmeeKMYW(u$t2+X@)V39Vn8cA zkWW{lz4q$F9%0L5J}AjPT5U)U}`>pHRP3pVkch7E+ z=Ky^c!O~FL@=$WpO(cW72gvMagy=~*`B-O{=NXM0Am@ffW>vujv0w%xnhU@VLh|my zS2En!SHzfgIVwkVcjCk_Vxf>r0Mv58dPGL$AnPallbsKvs90mm9i4b>>eKsjIH|}R zgkE+9G@CA>S)dt-Tz6w?$^$b~b4ZSQ#qd}SW(k$D#!96Xio zok+sMXOSe1^)gqBqhJ> z{jtX-r!b&invoq$y?S3cFhZ5#z3Pp(*GN?f);vya5@+j2*0~55hXUUvJ4s{ z_xC*at|UOS2o>3~1}W{JUIb8Iye;H(uqyCrpc5TlYyu^rUC90mJPJL$?)-po8e67Hx zSiYqYTBP^)dpUccBNM$H#HO0Zy^K^2Lgk^nuo*J)7${FjbhipYGMu*Jp*L)*_Xsw+ z_1~tm>&bvup?%n*#z2-q!@64Wa2D1WSi~k8e^=M7klvakCJO5q-E^|rX?pf>fpMCH zwcj{8u+#W-IIda>;!QuvX0o>~htlC~f z4OGH49#MKQJfOT@)QM>v&NET~1shxF8kfTy7dMUP;8Ihd#}AdRbL8)#-iUBDEAM*L z7r>fx*?ovihzP|VZN7c0=A{?<&a$*Lo-{vLcOWxh5F&VTzy{_50vNV8FHF(T+!3<1 zFMJwT39Npk3IvQkP@9t+olOV*-xyXHl<_ivgcrTpYXHXNAAEzGG1Dj6IQ-(n?Q+C7 zK3&VOJCyg6^Ks`FIfdkvmQwmRTi$g$5!I`|`K>#rwDDU)D;=nN#xv_C;OwxN=9Ub` zPu4f22_@_A-gcFbd1%*nsW`~Yl7e%3hM{;$A-T(ENp~qT26=_={F+&+??#kzsq}e= zYe~NNjmK%XjPB%*5Ec98;x$NW&K)`d z5=Bf}s~wxDO;Snl&Y9h;#S1K4#}45MWJf7XaHbEvb`eax z^F!_vFwfk#5yDSQAg3H`5wI3RKrB$MyciRC>IV$DUeRD#Aq*;qiSZ|Ik8G(P$zX3_ z8B2^!Z+J_?fhI@B@UXB1nC=G_Wa)Qm+LX04Da$r`_&{mftxQqE`!q6*r`_5~&sso) zc&H6>RhhWOX%hMGL)(Z&_?4Gag9UaZ&?G!Qhi8(lgx*1lwCPaNr!BJ8iu*RYgeZEq zO3AG9(fHe2GsbpGTr8e?JD1n*)D~a5=+!VdVf=uq4+>D`HFL)E8Ao0`3w60FOgmg0 zk>JoWS1Zc;aBAl4{TkwnP^7{7cdLoM3-|p+Ob9zwYM?BxP6NqrEI>u;2@|^zT`h1q z-Mmu&Q4oM7a_3wY%uM+LRe5l+V}OS0CAJ-c2Bg7ad$YH^Dh4xir85{1ceEY&&;JN( z0j=)LvClw0q-tV|UKJz}Ib5Jq*_(uAScbyg0|VU~?eM{opUH0&q%!$Wh>5 zX(Sw&^MAP+(bf->+eV8?3VZk4j6)Ba%f>2`lV9!_sq+X71sP|~pG0ExEvkILBn5uXJg&Ys+_IL% z@_Wx*(3cJjX_y2*^%L`1hw7Eq$i2i|LD%j-8CVtYxVc<)fZZ}q^squ_8a?yJrq+Gp zp5|1*&O{C+#hg_{YQob5%gwo$T3U|py}hMJmzI3XBQM`l%pe{DEx=Lz$Nne=k+WXI z2)z;p2H``m8u93S2Ru03HY(vTxU>M&UqobiD?ZVy-M3xY~dc%Hu4jU!B%IE+(PEXkDFoz2oH^LBD z5#uCC_FB_T=n)k;Fu{P%IgJ$pd8y=0!cHZq=D_9>?j*6u)RHNgaoB|pLG*6pW&#_i z+|DS#{E@Mb2@{B+nDFy$um=S__Y+cM4uzcCH}48xzxXol1q1YW5H<)>+1g*`T3II9 ztu#9iz%Z1Zfb!9Mj!>i>Ic8b$3UVDpVcxc@gXTx;&)z$BhwvKhL_{q}2=IRlX0};QF_+cxxWV+Oo;9E)VMqChf3jQlJxpu(6ZDDYGK>AwQQm^p zl5+N!o4qS>yGa!_OY%=HTV=7DKI`o6cM_r~Ai3xaRE?F|wd1^$d*@hkb&(DOcqXC_ z{o;HK1HjM`2wWgEJq-m|=n>;eM{<&Rd=rlYOqc0Eo%b0GnIV4rG?V{TLMX>Nm4_vj z|0je4p`fmMoc@c&264M}iHY^zjG%Xgtn_i;9}DGE)tP4a#KaqsefkQ{h2GowDO~4B zsZS*y7c*5~y=wH?Vd1*DA1%kZcB-xE$w{xKv)>5j%hnpS`S_PHq8&$t+agAN*VCjD zllCs{Z!J~CwZ|(s!hkt$^Xn2h>3}1|(8p`*&#p3=WZj4~jFe9nduOQWN#y*o3I`(! z@jp6a4c^ErTsi#OGU%qv=JH8RZKn{PveoQ=*|neX>G zV18anU2W36#H6IkckGy7@Ka0iI}hs*xtn;3WIEqPB1Pxm`L`xOUrkvIKhD{e}5wJn!dYj7{Tf`0>kpWV|i72;LP>t#ZoT`LY6Yy>- z8q&)cXmCWnsaQbqy(4xpC!R~;+7+mSLFCWcDW&*C5IjEs?JvLTWmgA5z^w%1EH0&XLjX232)dc<%U*XjO6-zn&a^=*U8 zXRPkv575ezLKew$a=mSqVC=$XR2soPD0dD?4R39^+K|TZ$APiuo*q!!N9XaC&eO+M zWxSIz^;aZp^x(S;v;-gGl0{aWM?;4E9k+8LNMPr!c+E*Y-v!|&BlRm|Fz_k~R`c>t z-kSs$DO+Rn38GXwo`g&#j$TF30khgWLk(X(-BrH0B} z?}6HY`J@vCmvgkZkllJw{;G4*E;>};hi>MJ zRS2iVlFA1vr%LE3n3K+j=1n+x?}>aS7LrfyVjg?#?m+)6;)0L#cHDrfOkXdrNgAp9 zJA&G+UarS(o8bpb+`}VTSS@_pr>@XPxQ}K|xP7W95BpSJv*lbMYl}`iHJFaU!IN^j zcE*l~{@`MTX6?C40iO0lo;KZ>YmUc(r!XFzRHra6ew_Df3iYYgX?}#<6Fj^U8;9z* zb}ztfpxIEP6@ocVd#j-_*LeP&m?48G3Q5la27ZQtB4Fh!Bv;Y-8fk$~#C~KRYXiel zd`Lk~P)hY8Of@rlqQ)qvsR}$S#6&S^ob&t}s)9n!($C?LP(s1!7ed5>=~_Cj$%w4V zJHqqML#))zmx_>|W-8vwP#;0;&K?4rM)X@K0dzbNL*^7RY8n866qMbz_g`s3%^ev& zlPpF~;7+x@wBPtA6c0GwP)b4GDv(X&w-K2-aGEvYtuarBKt~?oEz4=SiT@t@0vX!p zG6x*zidu_2^BGf=sZ)W`ztSmnN1&Z_BbWSr;#x zlYW#+=-_bSKtG0K@4SJ~^BdT2p@X{Ipa_OcJFLu@0`}3uinQBz3y|MH*IsiM{~U#& za|zH&CBeP#40yXcR1ttn;4)y;bTt%@%o+Tr~ps}|f9pPNFAB~<| z=*zaQ=yLOZ&mSuk+`o-NE-g7_yk5O~{^c-^ScCwa8Uz>Pc9~*1aqndQ z$addeMys%#QLdE^y(QO!R^+#>X}p&CdUgGCy;iHieP#yr32o}UCfj}p|CysL(s&(@ zjy2B9Go_LwEik7#i{%WvINQm6*C}ZZQdebkbHs9r+yw}KORD8)4&QOtj#*`hw7}*= zf_Sc$=F}-n4WVP0E;zC&`IulHyeG1W3nuu%WhASsIn!eF>7nb{ik_DAIt`7jxEBtx z@@iP*AZm1txnY4RZBa*9JS5l%LHmvvZfOL7&z@OAKNhU@a* zzy(B<2{8>pE+AgZ_c|^>()F?QWqbx5C<0OY_uxuD{g+Pv{f|t~eK|XNofep~N=JC9 z@@JmGt>=Rj6QfBXA*W;$9)EmaG>wPv_SRG+y=>LD&U)Q<`qQwqgHimUjJ>F?6Pp+61{yd;=={n3EgmqhjNq! z@k-hqHqR9lSnQ7>ziJ(+65*!P@RQ1X5IR54g;(Yrvod|GX2h{F@L7e8JkKI?=kXsJ zeytbS!Hvb{lXz(=h3`+RJNGSQ+?H{8E&TzE0(khiS%QjlloaLY`;F`iy4#e-i}l&( zQA*pz#^S))DY9DDxuXfS%L{5iit#y{&Rg%Ce+CSlyq$zT;cIXD$oHmppumbBP9&`d zf#v%E(Ki$FG!v~42)6zV=jjhaYH}-N+*@sJt5jN42nT4phyenyf`y4~z%WPaVBG@2 zap1%`$Z$Ny3Yw<8uSBAg+XJ$uKaj%W-yb}A$oR{phc-Dkt!WDvLFcajXGRFlgRTv? zA_Bb~567%VGMLde2fyl=BkcHJ?-w4_w(W0dTIs*S|~^^K#0Lg8#ICAfOC z=lPuOri<5>b9eH{*+tY@I#hL>^}+TFnP6ub`6bc9e)rrTI)R&U~{wQ1tI}<#%#J?Vcr}AJd z+6v~^Nqy&e@!q7Y2P*QLb}Z@hrYk9DiR{JO5`ts{@^|~{2Dr_Snz!Qp0>}IqwIvDX zk3h*upx({V@9e5ezetFwxOMRWM| zt2hB=(9b}dmWeA`*Sh6e@xKTg2CkjH1LS|WlS@_(-oPty7rhc)tRJm#{Ze8b)@4UX z%i~6J&39sZ#ALLVzRWYABdocns7T2%w5(@5>-dr&y@6F~pS}<4erG^7!?)mN&h?d* z@_wuN{;sTAKl<8uA8OV#n*+CVAN>4G6dK}VV@Fq_M3^HCbEytATRh2nb}QE=ud}mr z@utSyD&8oZUco58Sh2DWyxyxXF`BQAIf4NK96P28N!Onl;Ktn?+<(32t@kYC0zv+x zq$Rc8$-7kd25Wm3rXiUPJ{+`2B!|2dZ&}^hN-}0ZFOi95l#p_cNhv}p)!4kZfPsq( zdsw3HuqK3^q_33}fJ%hLDCqZ3LpD4|iX~1r4(Bj$-Uig$5YHVG6bj}f6SJgP8mA{B!58N7KZ_p3o&(q}rz440JkI#N1Kb13 zaB6}zaGeJ|fyiX8`syK3HRz{U4x(;DofwK5ot(*v|FOb{?ckdUwu1}PFmmnvi4GkK z*)Uib3m_Nby&etr=1`eje;eZH3R9lg_b= zw(5#fa9$zN^UI-K*4&`4c|ApPXGi3DMqg6pVVOa$y){5`Ij~-fV!gEr`T0a;ml^7N zqbeF$!_J!Sw_6hS+gCd~Ts-8MMXYl19uv})FJYF6v40XEdE3H+^_AADDCWfsj^_*W zTuE)#`9UuD=v*TU%F0T0`Ume>m(f;Fs3AyMA%hTiYHK5KdzMj9SNNE8_OuWX1;ERT zx2TX7ZkhLAwx8aK%s&Gr*J#V5b%nXrc`I6&9K&ss&2PG&e#)n><=&d{z$N0e;?yb! z9FsAKxgk?T>tC+eKg(xUxWR0mBXxQy{V6O%1C!hb1!Tv1tw(rZ!{~J+@K0xz?RmuSd6~LD8f+8*ojG7`sIb7YDJ{WDh zHofiZe8>?R8rllJTIXNde~dqNKIBSn!11AURnBizcxzzP*M{Yip~%!;RTVSDIzK(_ zhvQ10#m%p^AhKxuN}_`b-`y|bcEy{Of|y(Ix9lZ>4U^38Mj*ezACnOo-A<5*gN2CK zE2TS)Gd62??2aJGPoMBsw`q?(14d}c<0ByHk#e@7VaF7m1-x)azo*)tUh;hbK{DR9 z$)8YkqQ>Rz>4qTD`SdjTDEU^?Cz@_3H(7r0cG651~o_iK!9mV>ja3q9{ANO;JG zk2R-{Nw8CnZ6rSRy+tD_yrv#5*|D6`d*?&`+nw_QW4xU8BK` z5S$@Q!Q}KQ`yn`p7K}M)f_{Sn^yC!4Mp~geLcu-iJIs$Hx>>SM6y2KE&;C{RLQT{o zZ1?19&>{g#1n{`f1ip{^6;gOb(H>H08yWLjNwo-K&R@9Dar@1$56rB` z<6i6>b_3~{BYRIJ>))Rl6f@t5!S957MW5k@=xv*?uC7|McgO^yZ><=9lw)>HAGuzV zeZ-&GWk9pT*J0$Yq;T(>BYC64MVMN0$vNie?dEdY1$PBVLq-=D4SUL@f~h{TQU|zi zo0RNbB)g^N(RyxdXCcBttOGgb4mWrK)UG^H-ruwgd|m&9 z5;(xsEib!Wi~nzJB)ygTQZIqaVf_Y_D#!8;(|G=KLFDJt*rGk~?dx7~eCH{Xq9r!g z1DYGco!;{+hUrgWiSO00@(3BKBGdEruQ>@~3j1E-Q!beseh;c3^;ZFad zBOeOz_5FA{lKmv-&!d)`HaTDIXxVnNVHPaV>$((Hpq-uB&ZJ z9>`2ueOs;8bAv^I(*Bs?r1@8ZsQHJPKE_kZ7Hq#uno~fkXxa0h zc4b3{DqaGF?Jd9-UXh2R2!R@V*ja?;szSkb1uu0T-u(8~)}?+;#pVLlnvMnjK#4%i zt)#MIZOyE{aydVLwbIYjqXdmH<>x0)_6lpl)zi9ub)W27#DlKWUL zq&;2{5<*bcVSX7v-4QBuhr>7OHL#7k!RP4mlN+v{hWu-6{}=w=!ZCMRvKBUZ@GR+= zlkC5vb?=oYH>xA(XV3m5sjBFFV7be3u5i#@dkkS66h#6zUvwE}vf^JV1Je-fe_SLK zZrD^HPx)VBg14P*j^yvFpb!!+=Y$&C;Gl~8d=Stwz(hOb3>vD~7pPsX5l;eLVdl$G zV%pCOfyiiDP~`#h%A|kzyS&~$_}ADThkq!d5gEnDG$F`>#GekkeV@T?+wd40rMwsM>v{UV+je&u zks;q%!b;xx5ySQ@ElLptcA|An>8}-xF#Z3>Je}z;JmM=`M&v3fDYau)SPyV`$6%PA zylV+Yu1m8j$cj1Yjl`6vbrnTA8+}5ol!xwpB(=2?uMs(*0uGRS4@eKWKU*m#?tKGx zVCEg^4^w~rvhPlV1haA;=wV7|UZgDvr7T3IA)pL$4!-S579^d9Ck^Ucu?{~+3z!y) zSa`X*0@Fj+5&EzBh`13rX~-xU&)h>W?70#wMH?ZXT%Bv!GdezgNQ2aA`=qaY`PSH%Y$xU8~DryTF|dwM4WgoxP%B zeiZ*U;p(r*Vw{O6PZ7x^meYIBjpqcn#s*9_h*BWe#If_Eo9U6Jzxu6SYsYjXC^1cb z3@%O6xnhz^dyyK+a=^W>X7g#^TnE#t-)1ly=r;H+4btZ(mZW9#n82W_z4v5QIJ38n zB1uB8##g9=XnULB^LQDSkKdmCOz2q$bCZG2=3(>EIQ`}sLx#|2onKb*PKLDnMhF^+ z2bHk)7_`P^gcMWL#H=I)y(qrkfR z#ItyEG7=A=WDDI*(qTWk*&c%dJeVlRDO3!J>xDhNtEkH`B9#$5&Bp8*;diH71uyF_ z@xTNL9z=|aLdLCm=7(X8HBTPwt|LXRF#s&F`NN2xe^Dq4M&)cIQKbN}p&*DAWO zH$8ihVgCm(a2u(Z8Yl}afe!q+r zOue73PCqVsl`0C)@6P2DRIXJGL{u1&ijCriL2z&t%doz$N1@)L#6=&<;dg-674MP< zLSa1+jtreb zT*5&6xbg$?1@l6QWb{4oThkE?I-%wuE*(HBff5+AO8Ob^t;g{H|9HvSSc7@Elg+;2 z8nOe>E54;>C{t@S^$crkq0jYNgkw9{+t0mlE|Qy@VJ6O{@^}wR*w2-Y^-r&j=mb9(;Uzy9R)3^5lX4hCId^xL~9)ver8Xp>QZ=bf#|LGh3xcvS$4{!MaT zHB2cS&w(!#9xxs*>A077+c|Az7m5(5w?O#=n#{ly>ZlfUm}moDD>d@w2phzCEzxzp zBZ0_+@a0`nSdPwz1zVqAkKL>HPFz%{dWiac?}pI3MRSS zLqG*qzZhNxFfTwKRIVCms9v8!0Skke|8v3LM*e$vLC}W~1MlQ}&-bLFc(H*$QwGYH zcb{ooV8OMyl9Y6F%xWX$)!&&~LQZ=d?^-@wu$}r*T`-?k4AowE&RGE#T<2;8E?2u} z*@t3YjLaNNr*}^D%=4VN5ofv-PjpUUq_rY7;mGRm?rn&-!YW^RD5c2rlQ&lCV#w6B zxfu%L6S>S?P!ye*eZIW;?ZhP{wWtYNPl$_)EqlOIL|pXO69IM@aEEg2wAoKqx=0ho zoz>|g1WYiz6oVn}U`ox(ru@n-RcxTWb9CV!#digY^U#kyg@S*NgatY^m8|P(8*ksB z(P*zcRW=*_woRGZ=W>=(w)}5?;XdZ_Oq8Gi(6(almJTIxkxs1%Iy<+MyI!gih{@ZR zZ|rURkB(F;b?T?IsN1P}VrYxplQFZO*AK%y(;p8mx>4s_eildL?+Z{45|B7j1jVMg z>+=1*d|+;#a%f{#7!>~(5nPU@6 zxT@*{+%Aa23e3h>)$Wes?V-Ji44bQJ=_&3?LS{*Ds%pZ(=oa}tfd_%2l}{x_(M)EO zFh}u}jF;_dbr6`8fQc4N1}E#{;P4Splu(s`@xtFBb^>(_X1Dzao6dXWxnQAm@Z4C5 zVP}DTuVu-+P2+9yuklj`c>JaaytXa6dqe5-f3T&%2os&}EIV%)K}@zRN!2zMVmD?{UX845kYZ(qC8yXnl;Sz0@^u)tbUD3Y$7D-SOCHkO8gsAyJ ztD~2TBlA0^k8?UUik5Hi+389|XY=FZ(Ac04 zQPPW^Uy3K|@&(a6BJE`|6FI2d_ZB{C0NZn3dz-&E3K)DxWN|No;9GLOKyfdiT3^yG zQU=;b6a80D+~S}N5((~+9p|Zh3*OuU?j|QX8R+n236p>N$?nGc){3#=ptv7oIsh{k z-0fiYfV(HbBFx1>#?wDuK9w7NPuHu_+4HA0OHQk97*)kA+UMFMA?WN`woVi(%DfzFz}7JrH>ztW=v(SD7yO!g^HZ*=g%7LVFWvJH zxICy#Si)IFN#dyGJIr*&a{_bfBqlMkv#Z}6j79dOFdbFL0-?U42+qkw0hyk=_OTe^ z=pk@kMC?#t+_7*>*A>vk!5FqUrO31mD#eE1+Raxffb)#zlO_xx}hfDK*`Z%gUt z58GVnt}07cl(yh0L^(l7BsI3FtgJyxAMS_C>+d7Hu-q^na?EkdaH*%*tLbBUmkRHC zx%%_ALXu(`uAe^DcJjr=q60m1?xUMJo7(I50{eo|g+G*E4d|y1MilAj=gk@f(>{~Q zah=~4GAMp;i(7H}^iP88r!j(xOB6reJuIhQ*)2#bwO*>4w!kbq%oK$ds;`$R=XOhm z6<@z~t8w843Wa)(*Jqr+wRkL^Uix5Q;_>@bB~Ethl!k9V{R9t#0$sM;+e4+q?IQT` zjxOIB3C^Xy?=*GG&q@S`s5OS9#*S|?Ht}$sU(hORuV!OSF@LtWDOkWF@QwX;=Y*ek zPi#Uw*8f9mFY&jjkFJy znP1Pm10T~%A}Pmq&EseTKD)KAW+lsS6v&I1cPE^C^8L(A7Mn0ltG0Mee}1x4SV=}~ zD&`$R*{@%J^5w$V(`A0RBi-V=L6FNH*9SMO2Tkj(h z>a(on2qc5=x8>+}G_swBZyxk}vc7XJy{rv~-f4W*WP5*^Iv!qEX$I#5Hq%vR{jqTu zdpEK5s>swBCEJCNMeF%6n`wJK$~c?!rJCu_y#yf}+kNQ{_igQ*Ym(_KrUmcZbs2(Z zxY6+A9u?0{vv6))f-c3_==WiMtLw8Ft9^R)If8~l#{=Fidb{ts?+!(S9X4&U9vbIIU}=CTQAc(t+pIIh@nj z#KdRe5wo>(#Rti|Z2FjWI5zbYs9^^6yRr&bCu@gy4%IN9cTI$&`x20~h&=U^UOxT# zhrCfgyb%Y7Kq}RN*7Ub4iC?O699sNpbc5at(-5+Tauv7wboQw4X3M7>R2D=EIuue1u_#Zl#x6Rd;O@JF6CKJp2^i;mXz zO)mYvA4>>q?${i+R-C?nu=G$MV)Q#bVOfF1GMsk(#+_PZPen1-P(<}i-1zwjeW%o6 zF3N11+CuRDlC$KGEnRr{vpzP%+f>{`d74327%O3m8!v=;1Z zXJgM63JUxrKVB<>m4djTTrw=8L`|LW`-3U}=8ot@b{9mOjb;anZ&Q|Yu6_cmxVRX> z?bG7!yd-{94;pRXNCbEMWba8??{~7DUHeAGWBYbBHt1lRRq#xS$+0wD#Oj)0&!mfW zBYnVexMaiwi2>_{_V67iHP<{t&54a)F+`L->mMK2Qq#J_$}$D9$Z{%w_8=^i<+7R8 za*K$to68hlE^Z==y%Y0Duk*J?f0mW!a71x=!P0KeaL1=RS~ zJ2h*9E_LWOzo;cdh&yy9wvXKQ<4oWXHG0ar}MjbxR8a@>Ym^iq-wKauqka4cY5{gh$2HIwPUL6772S zlQ*tH0JBfYaBRNPEjWu(1hkF`wFOm=eP^ed!z`OP6v*vdN0b+l&p(*hWL@mkeN^u) zXxAyz-j49~jfjDON?TTl)k0ivDfqwjr$B4LW2h z-Ljus{fI>LPqRa^y3L{As_0UfBGPVe{Sa8x_bZ(9?CMZXZ|bDAnRGW0JlarKJDo@5 zL%nDwg&xRR>pl7vGr>PsP-t;2hIiS*#j$-}_q&45K^I*JyhtBUSz$L=%1l9yt8gRc$QpgIs+$YXZ2?s<#e)ja`kw&wK|s>dmIu=P?IJm2hMQlZu}Z+9etO- ze?HzmfS#~l z$fKz3D3M~l^d2s_f>V8b*14CzekD6g2$|8 zLObL0yiWZc_EUAW8>s)9B}scbQy zup)q5_oq__{wIC?uK0+-y(5gV-yu#8Bi=a`@8g@uGoU|9P%vb>u!MY9oyD_$A)|y2 z8LQP#AwJZ}s(=2v9Vd3go?h!q_Y?N|1=5q}rm{L{oG%r|6BikI8C$(mg9={fk^=k- zZ|{3*u0N~Ic^d-U6YMc}gI_1O=r{Y%pr%Jtt)^*!?jB#o7Pg;y!W|^@}hdouwARKNr$nK`Y*Z+q&lmzi0>J% zlTDOFM64$DIF+bg^D}IZll?M)e>PYGy!u2aDRsY2SqAE<$c23Zy&q+^Yj$O zKynf1f=z=RTaRM?;d9?SdLDH>b=YN8b`~7j5@&DHoT`Km@s>#5OU$G>I7()M`Dwkw zA6bYj{AE{>NB%WDJPIx_%K0_S?~iaSV;&K9c)8dOEyyCr8&z(JLrBwlPc~dWgtIMp zwOt}Ef*AgIGU=t91*2lITc7$`d43`G!IvTd0*$w< zM--L`{F!c-FZjR1^91h-6e^8Eg1m)&^*dZ3er69j3~Y_^!8Hfs;0=a5bvyo7?p{#= zsB^cB!r|b{s=eOeq5D*tY(m~08yU>|WoHXrO$?8fL5LIO$-^kPI^8mDWoKSI6c>69 zUOY(TFvC6=Vi*6>Z(CONLZt@5ec^K-#gPz)XU1)L>{QixoTpssp|)SM@nYn=+COlR zmYZo7od5a@1j$&y6dcwYzu$<|uqo&OZ=}{gz-uT$q>WVAI>91Sc^_<<6%)F?;=5u}5zEjw*7)cU)w$<>EYX{D__401pw7_nK z%)a)V+dRzBz8c17S8sT@>r1!aXD9@fJ?9Clu>PpHq|}!UQ#G5ocz8uAL+?t(0fSdo zAKx-7;2KzzU)Yn})2mCM3-dUfmGh~ucD8z})JzO#9B9F+EKVkm_#M|HSr0Krl*cvv{zpUjQP2fF*^bH6(t%+dO#%pI04B zT3)~pKZ$&mUH4wX+>z-EpGf zw68$=D?3oR_=MPskEy}6wp3u~mI;?r+P=p~Dn!)yS*c4p7#^XmWY$jxTGmWLlNa=? z|L5bKe9W7EaKQkqnaH)?yo+RH01o4HwMco{1hq;4gaL$5htK!y0Pk z?YzrP2+ZQEIe}Jo?^+KoFRm2hP!5dP>iVjrJ=5HU`(yZEfJSq7xOn+y&wTs+l8>u< zkW96$ewFi6Qgg7#y9#&DZQ%@vxsX%cZ{FQ+Quet6r@1?2C=#|SgE&Alp`RkfVLW-V z9~jxc9@Mp8qQg=ZH!r-)VXz=NWY&XQy4oBn!Tmnr{ktgy(e(z>pK@`Y(@Iq>eWv-y zXC-;vL`P@Xfu^H4VHj{QS#wxw*_~2cm@6nY14fPi^&|3^QQqZEpV>J0d%ntyqu)f7 z2r~6ET^+?b%st3vhsbKM*Z2zAL~~seI+|fu5wPiJ_b(kp_JTxEdS`DNqt(vuJ8-ex z+j1Ch^DH_0gnj9S3K?WpoIEVko-X*^Yco|7d|NIMM-bEsu`nQ^ED~Z5!Hi3Yz|IWG zM5qW2A^-iQqZd6ofXCW5ej>4=ACAZ)pAbFE(L=Li=}$u$;`&kvNWKPm<=dVuJVa#K z9)uw=gOL6Kvp5oSV(=}+`&?a-g*{b)dVZeg1U~-H$G(2ag$HA=nT>7dZPn|VYdUNc z(}0E}hr}F6sAf1F%n~j4auR%+tNdaAUGWoIT9>_MI5WyyN{4yfGm_N~k!$myv(8`Z zjQcv0={jlJUn;VWI5y0@ATI?65RSZwn=3E8%}P5k{U#oOB$KRpv!n*%e|!5Ri;w9F zen)s<*P9yw6PM15l9$j0T_g-#JJNwQ5?cMQHF|4Ku9YqBf9mwCIw0V;(GM0XlyVms zX{E)xqGB0O0PQ78JlcB(ZcC;6Y)tghGVnRB{)8I{K7Xn2JcMavA3uCJJgrh>TNYbl zoJ*bzK|HfmDy(0v77#?-aK2^FSk>cikf=aWp@{8n<@5u=EJZ|pGI(FT`>X=;fU1Rk%SMuHNz8vIZ2ef>Wi6bmg6_LC9-TUsr&%K!NB z;}l9o;J@O^^Cg<&c`2Q2Rc-4VupEh30oSs~=)Qp@AIb-ht__cGvEDRQ=`>3v^HR5Z z#Y%>mFZ^o3W-RE*ulORPWpQ_UY`Ks$UtMOvD;XR0eCuysaI&zGgB&Kq!c8Gu3NbZ4 zO>9US0O6~i@qhO_JWEQ-p9;FAI@XjYe~VIZ6UxT_``drNzD*hrx~JO+!_sqa5ePMpfZ!D-OJziDm+Sg^{|?jEH$e?OjYV)q&O$Ce8jOCwj7kaovUP2P#kz*JtxB^#*Q z0gUpG709<_Hdo93+gMcXBC5ZR$toP zzw0e@4vr-OEuRn)_OD4O>@3XePX=4n%#$^lw8FD;Y~F=T^zIO6&RMHxS2Viv= zKl8$3!JDQPFmIe!fp3**@}WxRI?XL{xINT@1-ca3Co_~PPN!6GaT<%%tO(KV|DH{(_olS1V(Zl2J=n!q5fDjd z=tAiqZRzy3J`^y#&OK5+HtGo3NvVTX&{d>RWc$?Mftp67(S`p#y??)bbrB@>{;}VZ zF*kG2eVjIJO@*MF2?&9+Y0XP^8~m53~P)x`Gq)#Z%g3^gG!F;X;imQp9^?gvuk*fo0hK#riaJHe67&{{TOYDZN$yiddku&`<98`!xvchj({A`~1Qzu(94@uSR z)bQaUmBQ`b^KzE|2}W_|03os^;MK2xX>5EZ?mx72`|qYh-Mr+<-;0Y~>$G2*={&tu zS~HInSr_<%=7)vDcyig7u}|G3{ui;|O0?&pmBrhkTFt**drdo6v5^f@l5+N4ztiZU zqTIeX6`B7^W^r&rR0)w!G-%4Dhp}AE&6t!WQfHVomy zf2<*REQWmfZ82Jo#eZ*7KUbe5m6E+w%lZNG2Hr|^=1H=MzX-m!d{l1roJVMU1TT4wE#afki+w===2$V19>JyKy6K%|=ic#knl(UsvUvmOnz%Fdm;T*lrgcS@vp2;d zzQmpE6aX#HQ+kgNGj8zl{XeX|cRbbo{|9_ZrBb;XlwBb!R93bIu9NJQU8!u!%%*vj zP$wgska9AgJWcG?$_s_x_;m5zW=&EkE<>n=X}O{yvFnOe7=r8ukOzs zk7|ac*T3LyxXuizW-1`&4HcjkrB|R(jg$Vl*1Z!G_mRRTxaH=gg`3Zy2A%hMTJi# zT&#Kn2%l=0$WCGyz_(-12->DmLKY&XU}2F#kbG_z}#xBeGA;wb> z?4d81o>2^p%U)4PpeQ+{xX^I>*l|2A3|UEV1a<;)WdUa-JJA;E6z#W@^F8jt!j>ya zn+DURM@$AxdosG;A{(%8WF=)x9QK!D3*-H44BzxS*MLAt@GzTOkp|oc8&U;`E9CpQ zC}jBc4pk$>ybRK(JP=G0Qqpv05z^0B#+8hV#Q7y2PvK<@1^dNKKcT|?@#9CAsiZHM zII#0(jq3H)1J#1K({Qh7HSomoHQHuF|D3&IC)RRGOt&dvU58d|jr=Go+)yMEF(il- z{!kvtfW_!p*M+#M3~RU-1Soy<5V8W^aB;m$lms19HvkoM&)eiQd^zvw=J6h|wECN2 zp)(N0KDIM)wZ9YzS%CccJOJ9DR&lj}GPW`1Ap&)PMaecSa!Inr7lcmD+_LJ*P&m8# zlbTFMZU~Co2NI*ztvPX$iznuSBs7fvxOhC?WJOIE*yU(d`?D zV7`B4?BmB%C3-S9HiH$J5YvXYk9!uoEQ{7UPqVYLn?Z2J>>Uo#mC$~hn~ou{%b#~; z*!e?Q(%-=mcj@&hMYX4XOtpLy&uKSO3+GaHkTfmMxtvgt8V8qU9- zEJZs#l9^H3klK%h9}VXf!m*(Ozqw8nr)2b3Cb`UxfPss~?Ag0l?Tr6i>Rl-J{Ecaj zzHn*NuAma9wyd%(oN3+v$96Bbr_>x~6Frgzt8;6JdE}(9dK;TPGTm|~m~4wscV%}A z{)(*gu9=%BBcSKQ_0{yP8wC3?yiu5fK~w4dl%@v&W$#m#AC6#3D9OtNR?;u~gYw!- zd|DgEj)oAu1(!F48W@k1F`H;S1^@+3?@jD7X2bz+(`-8_ENd)uG7?af{}s)a>&HX$Q<(`I}dg= zg6IlJmV-zNg?f(IZwS=BSUV-U&W~?Ci+=Z$lerV!`Ax_k*fZL1SkQrWl*SxKu!xOkeHmABJQ+P?H?8mz9Tw z8tf&HhD@LdlFZU5c>KLEVK->nUwIzuTe}>^hU8{u_YVu1xxse}eJdSR+lG>%nZpnA zVs5R7vl)Q_iF7mnP*z@E%XbjgbcQ6(X*rwl>@bHt1Q$Z)&7Vgx+feMyg|bvdE7d8< znkhm5T0ci1R7?G$ER9v2C#2VFrG87+bdFepg$Omnm6jt02Y!r6p`SL-`)92ezgJO5TdsX@}#pQ1hfA-~Z${|g(?cVrN$4E!RYm$RZrtcws>hDq_*{h@;m?d{zixiPGq zHf|J1!5WV5AG-4GOhRgDU<-^kZ9^@5?7jOp#=JT4zi^;n&6S>_D`+%V=3*<>z+{dK z8f$*bGJC|9xbnuExBZg;~SY zH1YSU-^#R5DmP|_8o(f-P`5u>=URL(^&V-(hW&oyugBeZ3b3u*kG3YqYcB`TeK1+~ zjKjA+fcV5;5iJZPE$l4KsB~f24>EMH5XBru0v0<2rE+=nY^jD;S+@s2PU8Ll6$+|SzbR)0)AAGGj z;0U@gz2wkl>da~L$0H);jva3dHYQId@YiK!9``%q0z*TSU%c42wdlYvG-hliTTtX5 zZBA0B7jb-%QAmy=cEFXt?{x(D0>Y>e1F()>1caJ1dg%JOUx;Gjd-w`yGsit(8V_y6 zVEK|eQHc_xj!)T!ef}yhaw+UiM*e5?Whg1Jqd%-pP~!5Yeyx6RBl#Acf)_E(h0avS!<5=U6c)&agt_DZ zV+C%y<-P&V(JHgXdy{cNzFc&7OR}bdNYDM)i&G(jK_T{ZaRvd+4C=O6P$@zt3|p+*(Z7A1YU>oi`K_5al-;8TmU}c7-5u7BA7O$ z?C8eQ3lAYkI(70latpWdz5M$}C-AGBfap;jb%#s9i&COZ0yEz~aj>H$U1l`MmUPeW z&qbGgS?_ZJZ?Z9UcOQc&ys`aWKN}hxY~#vfpIQVKh<*vlAkd=TLVygB!z$sma;U+f zt9O3mt3U6w5K|ENL*>c;!UpV(Dcbx@(vhjatu=#`Ow8@a8)kZ^l98tkfz&9}n;J+H z2Y9r}*%}+*0wuMPyvPWdouT{J2tmv?6vCyPq8Mgcdw-Zcvt}DA`qMag(2+Kq=r*IU zc&LkRvVHI(E7oW=06WPMN!ni}Ya?pixde+v6C=*rVZqk;lA=6q(S}b zI+vavVn;uF_6%8KrR=1Zl;BVio3!?J2i-!`SK#{)?%sxn%7e*_w#MU9WPQN)JlOkd z|HHuZg_V}dAV^KL1_(Y9Tql@fK0cgi9IOp_C-qKMoEB4D4ZJ#L)yi1DXL69bxRNBAm#<9qPMxlPD%`+7?_B66-qc<}_2*^|9{UbL~ zEclpA{G~r(JigC&UbF4oW$U^TokSN-?*CmftHx)y83IhQfBz=@ja+=oFv7X24e$kk z&cN0rN5mU9_pje$b+en3niS2u-)z1t?${x|%b{~xKxZ{yXdE0-+{L+=&$ABz^bmBhwh1%un11NT3)dKBrl5x(h1 z6xdI@hKq~qEzehx$`?J`cx(zS=FBv>D4yG(7L=XHVz#1HRyMjS-kQ@MTNx#lyo&OyVxr+Jw#f1Xf(< zMs;|@rRAxJ^{P$(+@Tj=J6J+dBA#p0P&cN>hK{X#(N>GMQEmEsC%zyDr9$uR?Y*`9 z{)oW%R9AWoI*;MBpjSCp!};Kvc$Q+{CJrCkW8OE78k-`P+$z+pI^U;#D=u1^Y4XC2 zYiXpI`p*g^NT)zQH{rREr9z@{x4IvPMf3Y@l~V%swJH;OHq$znMn#Ow>PAI=Tot!U zwV-dfnqIgV*(O6WsgY`!CsfkKG`m@KIjOZ-OFC4Qo{)No<*Ir;TbUy8A_IB(JwB$+ z0CzO`>|9v-%#=^l3uNsK<8uwwyO(T`Q-*7QB)uw$;vXNitPNSc6Ekn${ARMiW`@Ph zFzSKBkwUqq^(pQ2ZT&0P7m{fdB-2t@*)$|=@-<6v#xLvtnuz)IFjRQp9=b95M7dA* z{LP)0Dsu^<*W7JbR7Q>qqZVq0zb+qqTr;p)e)x00sceT?sKDyQRNY zPb=73lFG`5fK%XEpG}g#mJN`=lN9-hWBv{k#j~OHt0Jk(x(u!ESkd)seFDb|ToWeF zq)NLvh)BVL>6fQ&d*H8qf5+8aSc!K^glb6oixHOf;fwJq?X3g0!@k-kN5tAzqJKVd zFd~uowaf=obKRf#z`L!Um3JO>9f&6u8BRQ&CJue>jk|b*#HD0D!1F*RAfr&K&r~r+@^#>nCMXk5ul*Flp93XHY z2j>%5*hmA0qr^Oen5Ie#2L)D^dE<*4})+_=hr7X^WA;m?q-9B>*>8s!)o7oUwUHp{pAg@ zfV=NizPE)qcJ5xufzy_-Oh+GFu%hp{dia!NJSB7ZP_~_e$G7f-hmyr9_iJ7)Su?~j zZ}}T;>MgS?bGENFy`_djUX>*X_SPJS=41jxt{Eqb4$V*`$6{NvZ6U%i!RYa(qc4s^tHc4rQ~$?zS-hgVYFHTAKrdJ<-tKygOy%4 z{Mozz)RQFdHQvHEXX8eilU&jGyOjIs(k2hH1Xq~1fMK2K_@FEycG9!>W>UlI0u14O zec}VW?@@jOEMa#1G#7s%Ly1#xgKXaQe+m@44pb6kz&^U%7$mv$x6pm)Vk*dx4XA^% zZ?0DkBK*N$ z(OiIxM{tYyuKhw{UzTd3WbR$qNtA{atF%)Hp{n~MKE z+LV@lG3-iPf}BjdwbL?P*B#}*_5?y3;1bd82|mkxn% z9cNgj#JdtGDjchJd3`uEQ^N9L$F5!fk;rg*uUt2ISHm87=J;s@5uEL$X$>Rb&;AXD z;B1hSHQ}cH#8^v0)Xk-}x{3Fb@#f#&tHwAema$5d#Vz7}!bJSJ?Xm9^o%J!zIqn<| zW3B37D8FYn-Mesi%%>Ww|{P6 zW=et!z4cx#sDZ^EncJPrTx9&>%suo9=9OySSt7sww~kDc!11d*v)L40efq9r*_gO* z-x?iqH_%O)$zLT{m5Z>`8xn6&2zt!it?-x*)aMH}>Oa8F+zdOUsni2MJFb_``oClmr^_afibyu5M0DK3!_HI;#~kMVrHSUn;osw$pr_RV67h_me5OoyT+c-bJ*6 z@tpBITxHfC?NPDjo~O^}o-=xQKSQ%}Kg+T4gQxiGVjewona8#XzBh70LL{au64yp7 zC{YzjdbyEoNvg4xzjN(|3!4>QPPux$n;IgguOd!GICZ)6S`^DdU>vv>*RhqQPfbnD z=aRE&JGUpo;4(9_Jw4rGN-7>qt<&;*5Mat$-7?gSQgz3L{t92uZO>FT%H$XI;^yPwsX-6| zKL8RkNO+^O4m6}+)cIgeET@3eGzQ+cS;5YW@T{+0g>Yx-33CStF5Pvqo|4N2I+;9 zX>o;V9~X&zSH*dQYX%NFDv3s>x=*?ylL!3?G+|>DIWF;1LGaK)6|P)8dCPxHF;vb) zl6W0gPMYPv0>-bBfcQ*^2A%Dv9o|J~L1gyaoCZ!8e>=)8PLn5ycCWsI^T&Z3tn#k= zpI1%b2ehNx^{z9Zs?}oY_{^LD%+iOr#5U_#Q)_aM3z~0 z%e2^zJVl7SfQEH(xo@6xkr-PrUrhNz#d1wzCq)#jB!()BZnp0JJXl9s;5WeDz4Fyt z_azXQXOHqo)|E#@j5Un49z0~eLk&Vl(%))t{W#WB!wMWl#M(Q@4(w+<_mx${;I3bg zDps=YHiXu4MVJ{RfH_74=L@0IXDy{|z3Zdv4gKv$o%HHTa)iSa{a&iDrsBY}hc2xs zFt?d4x7Lce&ZX*J&ej6l8izjt0~zom$^j-C<0+NAEVq6^%`mY5mb~QB4}qnUXZF5ZpzRJhNDIs6ecNzUGswqQrhT_VIpZ-}3;;I_{ z;%|eVHm&0ZIC7yyKStOZ2LUW~X=Z95Cg#E9C_?Un>=8dU_K{g_!K*-4`5_3brgp3P zM=e~=-=&)X9stQIUf!5D>)>Fa**j1mGviTRv@?@A!DWJlUg$Lc@d5~d zvDb1wkr*N0ncnRt>}xsLfEV^G3>qk67QB^LpU*c4OAt1~3o$-Nd2g?u0VWBgpD{bY zo^wKw5M|7GBBsFj`(MT#klyX>LPoRgo}|q)t}u~WpLYjq!FDvmRx5NQS#=e^u5uW6 zR(LV^)uTMG?&%?Fm$6oH^RM@1v!4eA%zM<|A5iRHabr@zL^@kPIrt=vf%e_A*QZio zs;)4;MDE6A#*gbB3|Vz(*H~+kdcu(-N8-MJzrF$FNUhz^fG_4_L--XlRbO5sJdX}b zD0V^Gnsx?|LNo^SppBrLdxywK5h5xc=Quox_BX3k5Lg`Dzw;MFpc2@=u`72!3c3sl zpyHdEfDA^6LDs}v)8K)nsy2dVgFK#}5E^SiKKi)TA2v@vht#-}q8;sNRWQbTu}+U3 zjYRR3yaJ9*t4^v`(bS-;p?_F9#I%y41Nj#(wDjcmyrWAm)cKK@+p{d6ze_(%(6?D{ zVzr#;5GhKs@^_$3(ew#UYlXbcy-pNa`>=_xm0ujhi@XhJtA|fJfqbF7Ju{yX52(k9HvZ7YHVbNBFHTudGWv7w$ zhVssA_Z0Ki)cDqhv6eja{fwCUfc`H_^YdXEB^NGbKex8oaTQD-!}5+Urb-vsiZ8bN zSJzIw3A2^Z0h>DXm7fm2Jjn3IetKFuEr)#lD}W)mI$E{WSNSDFRw*lti)r_=Sj)Tg zj52U#T2;uj{crHfh*e2nrcKROiYPElmv<>p58=~}u`9T*w_yRD9CQ2_m0;hrrd%kO zGe6kUX?#_6Lb=S-1H%S^ylaN>RMo1FgZgkya3&%Z#nKXTzxn=lcZAT|fnqgWtLk=I5kKgAN^ z6k2z(V{fK{_p@Kti5%bD&v2-rMXRlMS#KbGCA0bQPua}o(+t3u3&NHD#6iJ69;GX} z^*3hnjWzsSU3!CBcx@H)=8ft9K;Xy26fsAReIDZi&0296E4GO9 z_sT}qt{*D)-b$JPvB#|~k&puW2Ap^$69OR{!)S`e>ZC{opcqm<^vpRnQP z`vRD@Qe}97)5$tlFDWUh`t#>4Cg1KQI;)LtT!c=$zo_A{{|g9nveFF? zW8$ceZmo#`kw2bh<5Ds-zru&jYXjnWaq9!f?REc)z6)r!$p6Qj!Uz8EkO;Lk2e3VU z^5j4AN14Zmj~rn`cjld~+d@C#>h`I3HlZG9N=OBDdh%oM0uqUkWxGo5{6zXG#n3nZ zTatxq57b%bl6s$Vy!roiQtKkbgVKFo3CG+ zp?;ZVLy+3~ID+8+S#~?k%6ciiM7ZV`%pahpr4__B3|+1MBYBf~+yX={q+k*7Dyu1h z9etEaI9t2I_sb5FUy0im8VaX=zCC4?t4%U0fz%VbwI<-f{X0RjD(xGceh%^Gp_=loRt zs4hpQdtnBKA-Wl>587sZig7AsAzn4HBeOp@s|G5=|JPuU2gb`ME@GDHueFNXjt;ju zKr`1mG0lA6F|tOp?-`?PbG#7`H|^TVO>3=j4Qh~rP!^G)`vO44h8P3IO|{&z?Rg@@ zcRN>iSD8I`DENLMiEs%{Cl2NCmu$Ch1Al3w+QcqgXy=elqI7y+trg*QhMW19`Qc8r zNVhVDS>HIK)7tc!`ML#c=~P8zE{SUj-~>x~w$_MOGA|bw#O-KcwMAnVXWNX}(LP7{ zgV?Ov-+u%~k;X{}QmZB~1I*jf$kj+P{TLO!_q`_dx735e@o_}c+8)bns$9*SBO@?N2$s!XJGcxSR9>crWOG0idf(zg=+@vUNBlE?* zs(HRLC#vF8!VGq)oZft_=&x+-?8EtHO%RF;wt_O)ye%hNH`_8zSI8pN;3Sx~J}92c zVi2t%xhe*`LLJiPJB5PkK!~{$e7~ZA>>+1KZBQlOn6+O8InKkGhVGgSB7-~cfkn=B9|(Jm?QQf>!dekD6Qwrzl()-j^KC1JfdI#B!rvI&-P+D%@LEw@Sv7K|%d97qopiP@}N@&_F# z1e1*R3)vjIk3L}>UHB_?NbHs}%D&=rlo!KRG(UG*a&qW?*+^w6?nIZNCh)f zCvFpF&`6V{h1DHt9zG>x-QB+FX=r3xAM0Z~!mytXi#peIH0E5gUR|(L%4eRRpU9v2 z_>RgpvW9eg1%99xY116dTZpw*TsosJj+q)!ka(4XX1QZtE|vR5`_NCn&0sC>7D&WG z1&HqGM&KsLwW{)NF!gyrs&Mq3Brm$N$O2f+?bjXA+Tp+6!SoM5Zr!J`KV=?zkAv1D*3fb*`v}vNZ(_e>@wJ*$1_rTi z+sh2nbM+gAqtE>sIP{%6@7TCNAFf4Z6bb~tYGY8WVB)eZO3uQ*um19ygAF}Ayg@ed zN`5DojS{lCuA0k`x)(caboy-8ooH*z@A)>oVa@sf@Rh{hPGOD?nQwmCKSJ34*)L_f zwIexLnUn5}!s2T1Mq-kjwYog7fPpyqEV9eJ4w=#SK$#rE=*{@x`*Ro{S&?Sy=4UVT zJCZ3m7m}x_hkKg8Cda={eJ(vxmlAU$`n30>hpMw%emuSrbjX{*G!U$jiZ*iml=j}X zZEvR0nwPZ(Yoy;l$p~Q;IXDABV@9SNcpb{n&D&=Br2h*L-a$>V8CgZYxQ(-$B{VZe z)CZMWP#Oj)m7<<+@uQJ51S`s*M4lo3BAOfhw}VA}1EaAhtXcDpI4Rzg5Q9+V@o;@B z1^W~wtc*|+({Y94QJTN+7dZp@U=JFro83)GT&`e`ka`{aE!11qufE`A%BR<&VlTsw za$9$Ndt2~v+uoV}s(4Cuyl%FD9wo-E`E1!+LUvB_jlZvF^sl*}(aY3|3ZB+a%NT!E z+#-~Bsbaq%Mey5&b)>Pw#^I&5_s=K?q>=Ys((0|M!t-VQYTMgA@r5dr)Rg#6 z68l}ZjM?R>i^Yq^xu*(di8MM(qS&E~VWO?jY?Kaj?{)N2SLn2D1b=~H^oL#`t8N_u zxK^yL0>(P8S8j7M171ycwv zBYxt?<}W{)@cJ3^T6C#mz+G=tD)$550`szB;RI^(RZ}r3W5=$cwegn1C=)#DamTZp zZ~J=FHv#cqcmJm4P_)IRy-sb|8-a{%NAM20LACR9p|X}K+BHt288NIfLdi7l-3l$a zfi0w^dqv5?HIrVAAA&qK96oZMQfVC>olQaj=OrfA)B>r~+UNvZ`;n`DZyEG{l07bSB}MfZ;kP-R><$1-V?p9j~M5#3$PEn`Km$ELHw+z(d+7ZP~?WYsn8 za&B1>J`cYOW7sQ>gF_}Juv#j2d9SC$th6Unn%;eCPTw7)o6k(iN%hC+$wO0mo95U{ zWG$Xid8$0{*+bLoqQ$Nyp&Z9rnN_p9FFllM8a&l(o0CC2~9YcK^W` z^x9iyzDMO&kpgL@>kOO9{!6=eQS|&Y+Kk}fW7YT^t1cv!loX%!-wGv-i63mUm;cct zc@#!Bnb-!r4zqAX51fQpj@DW%T}q(dtn&F)@Uy^2D&_cBw2L6>80Y}9$j$HF=@m;Ygi<0s=;@aNcYS(4FIuiQTA%K=ECY=jvo3OK3l?uMoqkc(5i1qoi z=TA$nkcZ^gUn(~{#`V(;>@z>a!#y3*6?p%7GNJwl=j1Iv=*p1%MAKYXM5V2TDxgKp zO=W?Y#ps^T0X=f?1({gQh%g-u+?)Qu%Fv`SxuEFa}xE?{V$@*67czQQEi6EN=D`r#Rjrbtq@f*y$=| z81p|C=g?-cEcouA*Y`|6J}$Wao_8Zq=C@AnWlOXdV@LA^rvwNf*JUR{w+)c_`6a}W zg+GjFpv1?Pwd@FJ?l?_2@m(P8Ek&8Uk}Gu!vqFt#4sIwT%6gKyb#+dq8E-zoLgWcu)L}#VQi4 zAQwS5hIxjm_nL3NGE(4D%=%9)!x1KBiI<}|(s+_UF+~~J!KI2S8)f79{DHbjNk`KJ zbE(`}tJ|~rwTF&HZyPJLu{p}jx!khjk?Q0hOL60Y+3#fU80!3MqW4;rP%v|7k;*3< z1yaitldI*>?w9P}=I@n{dYd0LyuZTBdwOr?QKp2#<#*>iwAd!<@>kjbtJ3 z#@UYWZV=2GOOUl2Yy|T9ikWen!Gb*h>$ZA_b#)as%1E26VoKlBGEgmUy4MVM6M6V>z*1^?YNVb`GEi)PG;uWM+ zaAi^+kK{ZMKO}2YDZKK&ReJT0eF|nE*BtBVG4<@1?JcE21ni*tx#391Oy=Cf?Htw> zP>B)ZCs1FU7}ij<@rZ~w(l5bU4DD8FBc1l}rOu=@*O(s(wjWKDqm>n!nmF$j@~>21 zCq?r$TgO@(nr!pE6k1^WW{zhMUr#3crTh@tY<^=Iy2O2de2P3&URM%yBz;0#V#D}1 z8?tPSpuX^yz4YSw25mLQdp$pj2#pXf3hSvn8zDf~ zy7fhy2XkwAM&gd+ugVXmLL0G^+tUotC~!PG@&S<@4Nb~F@4 zXuig}O!lJ${#b(m0*)7uW~O-3lOZN28#Z|Co`qGif^m2;>9yqcvSB*9${YY$TcFVd zB7_7Pp7YlX5PNUPU?+zzAu(=x74#69{c6Wgzf4qsHWko=?Fh)jL099^5&_k-ijqR4 z{|+;$DO?G16OfV>4j{b~p_?1Cw;?oX0Qn~xbl(ApG{a&oJh$kH2Pmv1&rhPE1qUWB z<{@Pw^nS?f;nO&`{2C9@i1BF_1-q0dEm$y^5a4K2*E(@cUQWx9u)7@;WO?A*I)jtu ztNzCN0rPJC;BewpRlzLe4@;i?md_LQ?ol-4*-mP2rMzGVw2NFEvMoE*01(~agOw(L z0!gy$BNV21Kn!?sxd@sAh(%&_P2E9v=sM0oJa_j67>OESpszvzAUr%jJu&1+kgv%g3Nu)YWCU?PB1z87R|ckkXE=0L*I>I=};1K7Dp z6EZ`ONl~aBVd6lk9*1;*D*5VCv%8UR(f)Ddefz?y1ztU;V_odSMY$$Pu$lec=RQp| zmHR6Si!Corfi!>Wjm4JAJ{;7Jy>FFYoS6m{)e)S>Yk8s(q+TWU9tM}>U%U!9YB_GF zKrK32vKqF6^tC}G`ip^do>QH-kNq6{$wi3igrIXTR!2(ms&d4ahRmon+OPP~q1FUF zKGSL@Ar0I=#vU1@i?6zvhxN)y!cIzQQtdz7M*$WBh=KE~Qi+zv=@r?nfbIHQbrV}b zr*dcCsEH|DSM5L$CNkDxvcona{eE`L)IUBxO?er{GoH#4*5m|GCIgsf^>XL0Ra133 zj&-YoqTV{y2me0lTOM@9I%6m;?O+>o_dOffvltF$_r%ZrIQoani*!DT4EbdXLa}>@#P7uZW{g7wX+GNe`38 z)#?m|0cI04mVF1oLgUv!ouyS*KCDmrn-h05+~6x;lrRB-4#gX8bpTnBQB3wWg==E{ z4j{U+oVST0fRlx`7GJK7x5r@Xjc4-XStpea`iy9McH!{4MSr`ZsfGqL;T<%qZa&l-G-)}6G;7~cR?CP^^EP1ip5jJzZRZuAkPrxs?|5LEdo?7nlCnNT)XAf5O zIJ1yJEs$+)(win+k{73wQcv6}5ZdcJL3TpW9?piLvv__lHi`llj&H-`m0NmBZE#lQ2Xu*!a3Ow)&kNZ$0zE1IJu(t0O3UayP%T|oZxDh6 zrX`MAz(Ik^r*bw~DHzlapiZf-IPv|#^S!I%zU#crUC{OIrhgOkijuGDeQXDF^1gN~M+} zM+@iy=b7lCiF4a2wl^o-7P@WlS+nm8;IMmD@ZbBY0$Z~#T%d2`kypsoS6NEWr*FtE zI5|SQj+c8QsR9{)e4zMLv3P#`XCace#eft|lCOdSqGwnZqHqviQZ6P`sKa^Qr8#GTg zm~LdbK?iVnZgHJj-!ld3T@;)lP=4+_T2ggw90*{R!dRI;NT*=Uw~e$=Yrb(&IQxEQcr480rFBSQC6AAd|jjyS;pu`AGNI zx6~!LXo@eQ@u3qkwtkVTf(k`?*8~&Yg<28^J*#>GW*8-N1v9Tc{OCV}P1`n>YJWiP z)q}O8kM{4*vt3u3tC(m20-r$8QI8K(A9g;XV$Ri$@*szf)gOi0^bPLVA&wG41v35- z_$3^=svuo$G9;h?jligwnP458;V72cz*<^+y7Xhm0UB|TkknpB-%APNg#;4Nglk~MdQl)~DR zoV)QZhwWBPgu4BA5q3#=0j*o+KX7U1s3-}9qEIQ%5$^rtD$qELQw!%f#*Vh?%GoHV zMC%Lgx3MfVgdSv;xn0C!&>kN|@gXyOE@@(*ji43U#D#QV^QwG79|Hl+-xv@E6|F3F z-7yf;IE18=<5g6;^Zh%=r21Ni1>%>t*+Uu#>61m*iCm9+=&DBBY{(JkIB*I=HusoA zCmLbzBe*O?00aqfv8&}*UftwOaXERH089fTU-3>a`oV@F33D|g@?J3Es!&^15hqt z$i_34WT{jF*Rhhshj%dt8Hn#XbSh{YRZnTWvRs_6A5XO##VKyLPFQau7e*;ig~`wx zY_gM?aWrG>XK$=j)mYcM`m5=l?)=x2E}(9bed5awYJN0ynUI=a_L$aUeF^=Ah8p%j zZQft$Si)aVF9DCs8Jx^1BUKHJJxQQE4a35g;OB#|&rf*mN9#DN^}r9qPU=_GQB%y( zk}t#UzSz-nz%ScalsF>6R6SFmnu)wOFcM$?A?w!kPAf%w2I9}r1f5RTEVuGH8P(q= z>H2ns>t15Ay0dc=4*}j%8(D<$2Utr)tU0faXx{s7x}ZPXDBDfXF{IfItS_zNd$Z2y z;~nXzj=e~(jflW56yXF1p6#J;PshQMpLDeZmPQos^5ybfvrP+>PZ&r*a4s7WIkRSx zp<63e%gxJ*Pf{)4Wx6ud`+RjU<9rvif#-?ZP0|g6!sNacZKMhuFBJ zMA-e4@}&9OJfw?XD2j>)1K5ZiQ+sxMW{@7#18k--HK9Dm>_AHL7nsd#axGf+ICXsA z^iM2eZpg=}WK~|7ei!&sfjLnZr!nU!k$@G4FEB&Y)14n1*iw zKB>f9$5eSPLZdmr!Oo2jc_(ZoJ_KB2gner%9M(opW%xR87QREzEUv(jl(az-%4@T* z19GCF$GuQCvu`hZM;gQ{CoDc}Fw!mqz0r4J_1NraQ(M0-DzC7yu`z3ql%5TLTB0vd z$Yz_zPxiB@fq<=+Z$E0EvAlmX1lVk7q@y-8kTr;v>yAC8NIzV*NVNpr%MrR4^kIRv zkf5v|c3_4OT%LtF0o0M`YSKwJXo>`F5j~)G#DRv?Cb?*Gc^2Ay``&|oW5&gorJup%_vbRh;@1c!G>$HOJ&_3>1ozT&Z z4Rx!t$X-~x0ll_m*G==Ew9L=x3PVw)b7lQ(7QjGAm|%HU*@ByU$NoRr5E>}NC@VjF zn8kiVj~o~9^?sRa+kqY{<1I*b@#>p1NioBtB39V7O#~FAf-j#rcAHc{N^7b~FHF|W zHV-klasmV*(rye1%Cx-SKnCRO*O9Xia#)%kwh z!q^#96Y=Yh75RIF2Rer7-6xKVHYqWX$b@)A>9zN<6{3G?ET_Id>Iw5lTNf2;5*Zfr zHZL<%$Yd@|H^cH-)e@qXGG8K%=F27K+b#OA0cHc^msA=N(F~Mew@40rMIGA_}Zv{<}0z!+<851}AEXn$Op9KY{=a(;E zt|%#;I=wHt%&cK$m;d`QkitTLC7vt-(pnR`Kgprm!ypbsz83)Lx#bg{@;%%@+-c@o zuKgl6>-$YJT0 zUq_O$o$MuG9{G8H81&R<*|Hr0IHgxwh`VyinU%J8RD5Z#9w&=W%N9;j)jxt#WKy{_;h%BvfLutDVNn5sNJE7$#Nhb}j|slo@U_%kbELFjqi`)w z#_kj-IXlehyMO4j@b}AvRg*z*i=T{rYOz95PWAX9(}@^*&@`>(PcaltRpNp*x_I~{ zCl1c>nu6+0A_(S_UdU1=W)>kxb!NBvpo`GlY~9<|c;KO>M9o);Nqr7Ny)Cj36wcPBvwIX!-psatk5mWR>roH}5-5KOr8EAF>b6i)c?< zNJmTje66*$NVt!I#~wG;Qv?`0>!xc?&M6m9{o4*H_QtC+k2@%ykA5|VTHsHy#*V98 zLa{>(_Ue?)D(SSeI@^cuKKwA?h=W2M*Up_gxz3+&Z`K7(Br+9kC~HJ?DQufnU{4x8 zXkI(t@L-*osdpEoB7)!3aFS)8f)UlL5YTz|(>X>RI+cC0JHyQDO*9yAay? z!W?|)SZ{wsS1u>WS(;BKXIIVA!FsHkMKrknx^ljBHeW>tKR%N?dJAaDR)hi_P{O`u ztH2srgnPoRyEu)@%gy(I1}zb!F0Re0@5^mLgMbJ*>?ho*P2tuMJu9=&Xm@eN`KkvI z7+$TEO$`K>BtEHL`*aH;tz5H4>>cQ@5`o1_CiLPCQsfm5o!(c9RGwSmYh~Ecb3SX5 zx(P~F#$nQX$MvuU^FIl1R9WpO@J?Kd_n(v7@IXb}yfN<8!A9O|>MANP>-!P3ujPaP zkE``6(AoY?eOLHSXfG>ij)hK8pQoWcXPL1l?#sHhd`06KHkqf&JBNOR^gS$r{GZ>^ zBt=2Mr);#?GlTKiUy#714C`y9`Gwat2nlW1ATS?KiC>stgCMTu+rdQs`5*2!wkCQh zVj`J|j z`knM~Cw2-^2&HK|%$kq4_D%5Lmk`U;DUy+G-v5*IDgtxlufL*0g@;Gz8=o23I<#hf z2^^m=RCFR}D3W7mslL9xnMxxgZG3@)kjjoT`qq+LLNtm7|$! zDt12SZ-H&^``r?V*hRBF(t>0~A8>jZveOKv0})FWX#Ws*hjxzerP_6?D`z4O`JM4vEocC) z)k({9@qcy&C4pS&{hsg=x^2A_%YDA9i0Z0}Wwrd6MP|oLeIOxG&Nuobfx>fp35S|m zoYvziP;;=}D)jU$lnP&W^5^>uBzNwn0Sv^`G<$E{Kmlp-!iSE+r+2Zm3K_i}4_5GP zcc|2$eE3-V^;p`HcB7%zV)o3Q1dql}jdiv#9;!-;JCIFq5^u;uG`E3XQC}TG~93)jMAQ zQm&(-A6RJBzcr!UWv=i*Pnyv_Mb*?RHMqT3vxL}M2NYbrZhSt!FxX5E4N~Aw9@Nk{+n;fz)yt~FYhA&b#z|CrqhUUz>*$Tg9Jjbvs_zt%j)vr&k zyE$Gfep6wti7XTFP>?+E#6CRd^*sIhwey048oatW&5!mR203Hf8v`i72g+TSF?^Y8 zU5kRU-B{ZjAl!&KoRSA`nk<(+wKyIl5%s|SObb>tgE;F`Q?kk+wteQ4m5@eL<^1V0 ze}Es{dEwpq?BaF^Tj-rb^}59ScRZX}_?}hv9%QZ|t9rg&jPPf#KaK@4tBOj__2~JVAdmDP!0a-FOv%v0 zJ*QIoOtu!;;K=1rBpj~;JBgxOSl zU`K*Tc#@_urvqAqF3JD}?HiuklX@=Zvp1wDmz>Ov5_5Q~f@;&7$6M(Mr$9C$TQ{(K zk7ig7)AMfP*A4*?)FPn<82zP{%-d4s`AA#<_B;Fjuz5QE(T46w4Xc~cOj-byO0(eG zpRVJ|(f1(@5wrdA&sx!Kix{i3XU`TajieO$iNi~d&ZPmx<$pr`rMQX^IQEYd$+ z6Y|%BT3avYB6u@60FSoHE4d5-`9_|r(DY|*#W_>q6=)2hVMGc9<9)4It|RApxqo&g zDTp9ZA>yxnGc2+pKQrz|ItqXsu-3yO2volb+nn&&M6HGa52>BvB<3p;ruO*SV34{K z+rRs4uYj~v^X6%&pSmC^_$y;ELw*$X=>d4@P&(%QFce1x~J&F_#IbWm_*&$iHufn zyN3M4$8TBgWDZ-GI!g1`+Efic^bt=}Pz}w9S3Pr@5rjs;M$WSz+JHK7`w%!;VPD>4 zZ5|nzpJky_fUGD*d{NCMVUtzI8cSRkM0w?*hI`x1<`96mPyA4l_w@F+WCh|2E4X}& zw-0RfO@)w+=I_e%{7%QuCGz$`Pc_9eX`GJhIXNZ2Q+m}cC)4p|!rZ0z!Q!@FH%DCg z?r`Q!-KW4$ap)i*S(c`XeM{5AtxGL&Ue(pfivNfV9$cE$wfSriDJ|WqXd7^0MpT!K z%RkWdd?g7=5kEz9AV~tpm%HT0BsuCN(+&^~aecwF(iUu!d81(Cn+b@vo5Oc1NDPud zB%OhWpc+UBm+?WcX^?Hz#Wd0!ABKcRAZbQ4z;w%n(@Ql-Bn}f;U{Z=TB$|QN%R_qb zf{+dKKz7$poK4Z-WbVuQ4c-4NJ!hJBZlst)LDIK3kfkn4M1A|Fl5V_0=Q{T_W0SCEm(DJx?wuQ+C{cgKC^{ZPVzM~@#bb`Lt51e=Zy?4#-$`+vCl4yY#6=WEvnqN1)M0s^{NsfzTj ztRN!2gpP_xZ$jt+6_Hg0lp;#+9U>+4VuL71FQElQq$NP41_*&~Uew*+_vf5FXLk)T zd7pV^?%cUE8l@`Be=YtX4C1pRKxof6Y+s!5U#H29vPLh~&)TDH?rOcFcO=|d zY!zT-;v|FDD&@RKet9%{8Pfm=-Kc2w;V_Nrc&oily?f<_C@+6)-)~AMX}biJp)vAR zXnq00wDw}KDpV~lo6mgi=}qYFzVN-nvFDE{6&Oqsb7AUy;)7viHZZ4Eao@8fTeuSW zbZ?w8-q0!Ds;R&J%fHW?xBEwv`6a7_SzUZ&vAxLkk zidS+!;AXMyIM=&%Q!XLM3=U@lZ#=rPn@*R^ycP!l{~ItmI7jyAWthf+b4@nVz!8{R5` z;5Q*^%4E*rhaD~2;{|adC zKLMF)zgPL~odH72#g|b2W-ro_AXJsj9@g}Cw#1byr&VC))}^Y*zgYIIXJkq*tv*dR z!rjg`w}2l1t)@u;gbKgL=!jp*AIk=3!_|b0&j5^)4Kp0LZkd>ivQCq#;2emTzUqge zS7D0Idq#W%^eVUC^7xkaSxK3AHd7|kMu$s$oMzv57IAZ@vL^&C#$2N9=lN0v&J3fM{q_%-xR^J-;I;7N!V+b*R^L;6df4ZUp3-k0|` z0S8G4(Qo2r-S+yslbxAiFnBb|s5wRy{b9d237T;kckO_Qp-7P`qwwmI?XJn70LQw9 zo^Rw*D4M|QLUgICTHA{0YTars^pt99BSm!YWqZ4xU$l5J#tI+Gl^TTK3 z;bz!t7&`#{-T9~?Pn2MEqQ`61cXx}YAg`MoW6fkEgRc*{Px+`4frGYOxhgh!V1+Ex zq-D2PvIYgfs0~NA8FYn-_guyKhEj;wGFK{)QAdo&S|J2ElKM?ycEu+MB`pt6$IP#M zP@;~kJAT;J3@xQFaLmFBBFx%Fzp#rFFj2;5zdW?~K00(8HZ2v@J|j*Sk2-0cx#8V7 zy0R^2_my`raE=Rfe#D_O3Lt|VzYM>$j5HA?~u)zt6}8b|8HZ#HWMYuii$N!0n`BVDnTXlFGKJrwY`|j zsg-lb)JzPa-QPa?pm5apcCS)LgI5qzt@s&MD+J8n%?PB*1oYH6)u<6nTtj*aRy!D5 zmPwr&7cu@;JAM404UeMpeg$oq){={}tL>VNA3*RXa9`PWWeLz$Ih0D9JuuvWm~6v* z4OLe2+#_ii{=wC>xi)dzA|6_Vt6b$fU|!4uBYoUsmm^{B40dGje5*PE8Gr=ck{!qh zqX*;2I}Xq6K(g=ldWlU;mF}Z}2RS?duGM0d4l?cD_Ek6+$nCO{H@1{^P}ZN@z_%Or zbu=Unf}X0tF^d~0u@uGfGF+*xnaPN=)4-W}fAbgUlPF4ow0R1*Ll~4gRa; z&wSKHImk=VwtYGbff3@PYPG;H(BZuIe4sPG3$P592L1LokhV)ZUzcB_Rg7QIdS^oa zB7=!4sny4$-d$WJoZmD7rahf$`;M#_7|Pe3VyUY3>Vg(>42Wpmxt?hsqh@Tw(e7N9 zByQ$4s^S#X0-=`%#mU2;qt)zjtz0Kf4^*DG)g)sVhNA8TrCi65qN9>Z;eZv|`4X*G z>BMUSu_9xk?7o{1u0PRe`t_Be$oLbmRq4JPdh zsgO3#b$=Q-5@0OXypL~VtZ+fIoe+3({ew$H#(qx>N#iv zx2gPnSr`5y*|yXe{s+Dj4;r&={crzPhra#Wl5^k7w4RXd-6f9ilpV&siLGO!e$l5d zD0r0QXS6i)ul=and)hlLLmYIL{h(m^(bC;DslyzY<|AF7~N=|I%K+1KRh% zebFsOp_;Xb4K9qN9sv;bkx#2)j`g>wQh`l+D^k-uJ~WvldvMW9+bwaCYTL>{GK*j| z&Kqbu_32gZpjXlMwMjzm=4~rO-Q9T!>0eqn^n(6t^V0l$2+Tcc9x(wtWA6Kht|c?i z{cjPsEMR=I*n{e#)xEm8nRp@fO2!4xKg^PL#_W08n6mo$#;JP)H-bCzyXC&~CThgw z8q0q$a@85Q$44FrB;TdA>(kSWZBz|Q^AAnMU~U+S2=TFg3_j9vtj$(|ZH4}>#ioI+ z;QP1xGP7mIWwNe^knLXb4HU7MN(%p^V5hF3uh4K~4^tE!}Q{ zyN#Is^F|`iJNB~l`&8cwcqgm4euTr%#{6NoQle&XLM5NV=F0nQf=AcJ#n{?7^fguJ zFcT%xmlApUe^|21NW`nhOS&rk0a5`bCSe#Vsj#;rS&?Bsm+&Jbj7LBDp}xUX(V<~{ zAS$~gZ=9}?nA@hz(CFq6c{QHojJ;*0{#xHLpI#H1{NgBm{;`-0=bq^Qag$v-uZyn) z?QLAD|2)5&qga(ee*cp)jqmD{V^JR54Sky})LKMm@yRLfUF>+Ono}-bCf}ko_*kL!^q9J~uh?J;oOI0Ly z2v2dFOQF@MOiSc(A^*A@il{^r-{ORS*UUykiI#&1|5J(6fn`e~+=SNX@HN zEOR-P-*m<+*f?V=(I0Kj&GfIF7fsIhJ8E1IW!y@(^Px+fwBmB(T}aaDRKm4u6cn{R zUnQlL}X8t2WkYdbci*HA-n$U7)a@ZCaYC zV5+~mLe5i^a!#2~_!9MpHl<~w$S5syn6KMVdB%3~*g08WJL!ankDC7UxcS?GWTnnO zWx|*mBxCULIPW8O9ZHI?yBsMnP>~xFl#yH84P2uO7s%noXbskbuh_Aku>b|Jx|f7XsTM@ zUJ~mXx`sZ3cpT@|{;JKSqcU^P9BVAJ<3n8~u9rw!1UC@9!eH{47M8upgOa~gYpI^u z7>6VK8lgf&XVVk-Y^LMA{igV&qRcfj3(FjalOwxc_s11pWmZ;+Xx<~BN^|$C^m?u* ztC(|uyn%6gJv3lCehEPEL{nO-74IU(W{Ju*%Qd8QU}!MZ`GTO`zr%jFy*&kA5jCvo zZzvGGVHw$N&Sjop>UJyTCnGQK%cy^Bus@yr21fN}!BG6EH8NJb>iF9aNtXi<1+4ql9A|dD*PYp4!pf#_ z1One#ajZH%p;4%9Sj-=m)AZ<9wa&BTM!l%$HelrzH{em;M+ixEx*)~kbj=?{V?M5( zOcYe{CiO)hIet9t!9}|0Nv}A9XI~YSo%Xob{wc?wxeR{jMAXD+4ebCazihbk?W+xT zp9Qu4#dksd9$%je3zymYjGqZ}Dcc-|#kjSKhJ8M#R8*d%E_SB1ll+^z_1twg-0*^Q z*-;D2JC1t)G-rH#^nzV}IrHh$XKfG9CcHRB^rGcI=b8wRr+ZzV6S4St;; zrA%Zdzt9id6~#yX$HSW5q@&K+f+b>}=qxkA)HSU7d+>n_e*NmQS7HcZB@Y_j62*Vt zk{E+KDF6D!Eb|M!>87!bLal?Qgt8I4*}-?_aJ=f9yw0SkFAGv+>zv%crur$T+83vy zUW#EuZ$Ue)R-zn+@n(vl_qWeyk+HS+mB#o6Iat_|eP$D~KDP7Z$`pK=>UyVWPU>TQ z>9zA|#GlINEjwCSSlDjm4Lsi{NHDJ)v{;WDl(Dp=Xz4F=ISej7FvEK7R0f{pRQ zp=GGOHX%R1XcPt>pLA?8bH+T1O~dJ49JVcpR;~Eqkm#0r7J|9tHy(q2K<-8J<>` zx0!pW3*+|gc6wb!Ki{JFDM}VtKks>$$lVQ#gy)xcgg6MTkFZ$C72I@$8yR0!v_v~C zl+)%&0HiE(=KMQ}ab3R$E-eRR)r#+?+1}88xI8PLrOe9T^Q_ONp>{UehU!(Gs1Yt^ zW#8Trjmm;w#B#@Eit@c=!vpVOePv#PD7SGm?EdnCYBYH@y$Q3sgxFdKA6r(~s&u1h z1AEksnBo@vgF^O+o!-`8%*c~>0r{3?AsIYL6u2tMfv)R6goFGBJxl8*RC>0L&Z4BmVr9bA{Fq-@OM zA9dBq*Zp$Uz4#^WBhj+P*UqYF$BH@poELHC2g_^}uVU%vQM|WXPK^PWl9=OKl~|gS zNYB-)O4i<_Bl|hnQI^y`oz-0p720R$gBEpJpBQW2eBU#(;hQMsKB<{2xo$}}4;iCw zRJNp!!}*Ai*2t0JQaXX|Rl@Q^viMTxu<^;j{lF`m!iNZ z;1hA3v_Y8`?nuGitrw05S4t)i_kbT#DjOvY!4&3gYHF|3B0qcBOcdA)m=`wk+>try zRXv;FSw)a+mogOatE^4blia0Fhk$`ECDRO_IlpO2G6Sy^kMeT~ENp4j@Sa|FCms6c zwCr~O%)g`0!E3;CzB&@mzNwleY!%4N8c7~-Ws$-Z%m=?oIO)fMK#9 zn3Sk}xSwQIIA&C_R9#z&>hjk#DP5ydgjCoiF7#~m58r$U(_|;XP)_STcBNGKr2MHI zmM1(&7Cep4C^RvR?hi&!br_x$i)ET|mwhfa%(o9*mR7EdiQKS{g0|(G9#m2vCJ&|n zd+&O5da7VgnNNQ4E(Wi_$He+YO6LBls6Q-xvH^-HGWzF7qva=JcL z?yg^$0y#I14si%98Af$Nhb6nHVH%X*n~Y>E3I*!Ww^ znPi5$yp{fSMu7pu)2Bl_;3L`*O?zw$t$Ut9ov8Y6usH3EW?gV3=fL3Bl&_!lRwF6?9>-p$g@^yC z5|wO5@G5vZIpbPwS1@bG&;42Yu8BAbHJ0-2wnIy=_KZvplax6>)ZNZUP_*&PoX<{p zI@E*JMz_T4>gw8?SbV+x!jv9KYYvmYG31|Of_Y!PDgQmTI6>ZQw)-_a5TXAbNXgCB ztAHNA<+&#oGoN3$`Pz}~j$V~m$l*UrS7_Oa3Zp^E2$C(iaqE_8$QWC3=aJjRQ$s$p z9lfzmITy1y05Cu}kNg>HSm3`P-s(@^pJZ>1mGDkuKE9?6r-V4TMm;;k`>J$2`aXxm zb~NcPeifd_2XgNXJilBBvo3U$#`P0TALKG!^{=#u_N)7Tm8W{(;F1YTdy~P&Y!&6S zihJ3cLb7*HbWN7Q6o)t)K`&>z==ecR*KIzOjR2zx+kE8<5^|2tHFodEtTcs!@71l& zMYnTL3v|xRC73>nA7e4vd_959q&@!gpc47_Q~=0QkLG+aOrDmp%BKzk7G0vR*dmTi zp);8CV17QgqwY99C>8uYMDWvkyX_K9L^MpLJcLUT1F*;1$+GUXCbSr+Uq7zr&}%rq&z+VtfGYRs zkNn*2{*ODL*{Og7xHX)(%=UKV5aw43i*_+|d&^nNu6dN5GnIere_$e-Oyy8@qkJkQ zIW7?D=HtDmC=;-dmJNpIE$jJxDaLVe);lmcl?EkJ6Ldu-P2*l$%f930qd z4}b6A8YIsA@j4a5k9j0{t z^79dN8k2=-gI}ibvhF(L}Ne*)OMx-h{Dc+1w(CrYXT6v~EAT8xUYSyh|K$t$OZ@ODy5dgGKp`cIW8FuGjX0m@`+l zAjeC041ivw$g=t`#|56L;rREMzyOCCx(H8pexK!zjNc;&;9GLyim%V13~iS*s+4?R4zCtf24RFG z$i9qiCt!6IPyo6nCaynH&SPjZW&ITkmmov=Ic~=&9SNJ)xH1>TQRK1cmxv#Qpp$Qc zD{ZjIWn!z6nvknoWC^);JXi97{P@C3D5uo5oaudo$seN3283qc`b^A}u{Qw%9>Z+{ zntC?I0ikS3^xBQALF%<3Evy5Wl&2es4okrzo{!>(_*D>_^4Po_HVPoiz|wgoq3W#n zw8upVSW2ftyRqEfT|5#q-WnwKSz_7d^LzIe5a0LCursP~Nj@)+UhGPvp%z747H}=)ai5X3MF6~ z&(!h1GOO@3D|3Sk$Us!0{^;-hqa9bU+0f2|JBdk!M^kc=ZW^)@cB>zr)Ju3H2#A39 z^AR*+2i4&Fq} zdqmb}vW?YzimGZRvW(lc*}G>Ip1kPtcR@w zGyYb4qrk+{>cvdN_YW!FLCF(^%R?IQhA`&tKO@aIgX|M;adi)4+3|O%E_zeJ@(epR zf%$M*|U=ybg-*z2$2w2IFik?h3#cTEyKOtDFs)9qmM20zM0iS12jTXGFxZtSzh5Q zt+MoTC8CH~xQMfYf0$%}Va1nML1b=^f>9#eAP*IamFfEq1&5ubOP$yT-J!3}FHw_= zeN5=7N#%3L${BytKKIXzJ4a?^l^Xy#$7N~|Qze|Klc|yTLU;8lajXHdL_j0E7l7P zzJ~!21f3D9klU1AEvtK-ns_Fp2NrcvaLq@Lr%fdInY`w9>UhyVubDt*Gktg+F$6Btj zK^C}4T`zTljQeb3534z+*!acUD;GvRlTN=`p`GV{<->Vj=y$92{WGP$^|r_nrhU{B z$k8D~>dkIyU25ehTot!FTG$~2ypO=E=(}#aDov&bqa561=9h*LG|bUcW9$A}pof~y zA~liKwkxyiVw=XyK(>lvxgw8%_vS(oJC%IwoN0IW@E6ss@n9Kmb}@X0Z;V_T(hYl@ zf$we%v5YV_bh+$Jo6#=E)W3L+0IZ&Ht!!;2zNofnxCa+9H_i&}x@%jB$^U#}+Ct6U zXlIpLsW=PwJs}Mv=d_ZEj_zzB1Z?LEfCOQ`r{y(5*hO zm}I!@NRR+5ZdBx%tlU=iWlCh0;neBS@NQTZW3o-*@V^5AM3TXJ;DvXiT~wAiC146- z$oThc-(pp3T>;U^p*T*dsV?9ra|aV>Y8ADD{DhE;`Z&8zk&`39#+Y*ICl`k@jj=}O z=67qHoS}E&Qq3c}zG{!>waupL1x1Zwi_TRaj(=vHs&iK3dU!2u7TIzo(6Cpic5C`p z5*wK)ZQ|}zcIE0JATTtq{i9a;HHkNj&s7S&w#cTUt)OrpU%Q-gn{(|O zA7-xm*3$HlDWGB&VqGE%bu=jT@2mCoTw(6~yu3Zx>kru{zDcyB*AId&4 zZRZQG3Ua*{6;-1QYw>_`E=6tH6)aaW&$_mL8YH<6f`cSf?F}KzD9?wo-6H1Fl+IUM zh4Bt^Q2T9lg0S?~c$zv)D(YYRoY}B;_^bx|f~$WWQQoJ(708VKQVgtg=g-R}q{^5g z#KHQy)2Yv%2XZUNaU*Ltbj3ZWtCc|=m>LgO^H|bGfuJ82pl4GmQ}7VS)yESE`71#sDafeo@U` z4rYeX>PK&r!M4HD1%vhcIHNm?R1&-#-OboQ(@m?re&+wBYuR$ z1<&d^9UWDJKZXj2D?i_BHtuFS#f`F7Tx;MZhlX+FfnZmkO}NI>7vBG<+}yLGBI?SZ z|GR1tz_vdmsW}ZGoavU@g%ety3N?jH^-bdpq{#^;BCS>{%Ok5{9F$ygO?D&2Z!_-D zJ>(4$cZZNaVPA_+0jkmGL**~AcVdS3lg*YYnSlL(v$Ht*vxSIg>u)D!~lqC&rKVxmv9jm{Z9%Xw3|fSOr=ru;DX) zJW4V?TE+t&dd86W0N-)nLWr8zUyF6x`PG6=X*QO;)_WXq+NS;6X+5*1$NJ-Lq}6n+ zE(h~VUhD)!)_$N$@!rU!vP4t+>XuJ~Mq}t+;Qf^kdXB7DEY04a{aDB65;sUOXrmHO z72GW&6p`ATv#~G_{|6ASZ4|{601Md1B%oxHXlBfKM7UI^og8b>0t&cHz@BG(v|Mnb zoDz%a2n`288|4)O)-g#q&}>&CY5Ekd0gmZyC7*=xE%!dPZ{+t(@#rqU%{-pyjm;O< zueS)8ry-nowz}otTGrbZPF9X6ik>-t8A*JC$6LD}eRp{j@JCXVotb?5uVU$7o+0h= zp?}=aFJ?Gcvc((*Cch>NE%gjg*pgczD=z^aD(p?l2d!_1L-(o zOz{K3(p(w<$mBU(Q(=oZe{TJ^i|X)pkRUYnF6!gV*UP!YC4mAx9mdUP?ED-;@T60$ zys*^x^^Rg8{!0ZA<1`&w2h+gI;z$>aSs~;vFe+3HC4Nmh0i5L|BxA}!Uy;PR8lX2wbH_%7$eXtOctDb_8uE@i+0z?mdBJZQ72kW^uGPz;l=qbHTJ_yA zVI;3c$%2P@h7rg77&+Za|K|LoD^i7Y^r-Dlb38zONJ*cDaoA6nInY9F@dA(+-I==r z9Oj*!cd7ydZEYJ*2^dzl^$Gz9gSuBhUFK|KoVozUE3lP{XSp{E_!wpGg_K&vmVK6Y zQGP5#k~&(++CO8ohG*d#h~97sY)=+D@1h$rGTi^EaVT}A{2k&}&YTMCgqMbGBP(-n zez4d?Jon0Ug`CiVmKevpf%&i*vk4c8>mnBJF>}zP{?Zy4@wp9zABK+Z8{xr`a|Y+h;Hym=qt3We%QJ{m;7sKE6Lz6ayJVlP&++Rp^Pddy!H;eO#Nc)6`^ zOpZG2mr7iFV?^S*>tN-;`A08yZxh+~i=;Anvf!5rKt%L6UuOMs$eSi>A-P&={7m6{ zJ(u^*jKVmGB8;U{$4_%{YGkUto^IFci5HN#_PLCByLy@B3L0u^<&EA-`?$xoG%yo# zzyr4y6lq(Y>y$l4;wsN?gsK3hppG{s8c$C7_Kh8%69#30kT!%eN$^>7Nu-2z-_~mU zUiFLs=n9youef1I95^0yKG@4Bk0@FL^FPas`I%06 zwZ<+$XX<)6{CQZ3y~#^COtjaRl#8jV+p&TAO_`~~NyUgg)4x5EB=g3JUD3ihCk$ES~5$8t` zb!COm0@Q|xsHHZJGgzo#;u{sDh6Ii#;0H`*Bp{;d?Rp{ zZdvz~eoQWHAk>8xThsLhWAVmly(>NR!SL+6{84@1Ycz!PIPDzrk}}=bKtIIfEU6MN zrkCBBK?)YC{WWr1VXwu%%Pha%W)1SG^4Oc}?r)yl%Ht*Peq0S<`L=ZvBpc&ec*EG( z<npzj?xD;|}U#4Or! zF~viO=ae^~xI`L~p8@lV(B^O(W@AP1jAiQAxL~@$0f#p+#HxXwmq4omv|JluP!2ii z#!96Z%yQ_>)b3~TtL^P`op?6(h(oEXyv|3Q1SNeV5r4-hn%O$_n~PVwY|Mzpv}JAs)_4S=FU}@Y%LM=y&6NOpWRuIX^0qx+I04>a@zK)-p9^=&l?( zUe{YcG6{q!*=-#XC%qnG?}Mg8Pb_kYwGczv)d;fxR0Hdbut+@<*#&$vP&LLu76p0u zrS~{6g1f}T{v_8B*qn`P%9M7W3fb=@evM&J5JqKB9=Ej{e!wj)Mx7q2dDoDy=3G}{ z3pIK1#p220 zdU|@&8SGDNt`Tl=N;qo*1nz+8zfM1w?zT)8WM~Z#qXKGHb zrC5Ka6@blcOSlJjG+!lGT>3eA>AwEh+hn0;!FNZ{`<2Wc0ZY$9`37GJmDt?R0a?t- zPY|g?$GAYA*zx1s;=PY@yGo|7)-Q3Vv5~sonOp2OLaTZBC-d0BRK^dPp6IOr~@;3+P6#epdu?&#?L_=3dJG-4=h%J?aA#dgubZ@FVCI|6PJyw zf&^Ucl}LB0;MaRs8E*Xyf+jC-uTeW6yA|hV)$Lm7AeH!Ac+mNPGFRD2UaVRW#;&u^s0kW@c%r0r+=$ zQ6JuZ`l=a>IFH%)n{{cMcH6sm$#cdzQpDS84^)sx(az5$lN;`DYxQD2-Zo737C{YM z67TjNh9IeQ-~Ho{TkH{$Z6`$eeZR3;NCN!>aUeX+TwX*2TRX2E;r6GXs}u~|Qaviv zh5it|y1cDWCg*W1rUw7Jc?I_v(uk+`xV7R7{(1=|gFRaPx%~j3zIZK=-ih2Z| z;}u-?Bx*kRUb#C&Xu>gf_91b+Pcyl*na%iF>kW86MBLEqwQ>>BJ3t~aG_+C+aKUb| zf>UNO9Z$ZcIGQs^4dOoD9V&Vuqwdhb4U%&x(4nXbEL7J3Ad7f1R$5VK8|G_w1ld+U z&TRFyL$H!#o6%Q+x~ytQW2;*4#j#&%)6iG#+>IWLQ|BpUepqTa;-Z=&isc)cGou?yVaVX=xb!D&}nHC({yqMQs^<#X(-+~|3 zedu!ZnikJC+u|!LogxdXT{;=M%`$0sn=eWw@xO}hxoP_P!~071CY;x!@5+juq3tA0tyLL3egSn|U2YX^d?@G7MFjaK0P5ySa^nW8 z*(qf&CF2nl!|2kdLmy;I5hxEL6SB(~RnQG}tq7s~R0ZMV{I z_!bc18fvbw5{{oja)NN)fGY5m3Es1(v2e|%){m`8U?2BOkanumLj-qtiP?!i zVy<+`sQ$u(S*eL373~J<@FDtwpJHaI(M$fK@N~s^)@}AOYlFJPVxU3IA3JXX=`g4o z-g?T~DcMGRx#&YRi;}GQfYA_)vEs)92LNC?%3>OL4HXMYSGhE$J=d;B)W89CRhN5O z1{7!5IsA}sM4^dw8M%|QDrb-YXWU9r>CTF6uzPp7)f00RaJa0@rpxG3KyikTFEfC> zOQi^*B8RPzMom0_{2b1Ij8+C^4y+LSU=`)MHW21YH)R3IL$uAi$4|xcFo$q!&n)Ey z13k%QzHC2Go7#w(xmpicq1_Wh{W!q!T`GpY@_9UG@k-$JCX)M??38{q$8Poj`mns; z>IRMSMX>$@%&Ix9lAfLp%%P{(NImP)`(Pnn8JWSKW6BW$SHpadhFre;a6nM4NX&^F zvBG_$3=?%Ux+~YnR|YFJ0^n6h#G1=9jjhYjAk>}< zH&W3Umsx)W-yR5RuV(^?oa~uvrTnhhT0_@oKJEY!5%yRt&qogufxnS-qU)uY)2%2f z=@i?@S8&+y3s7TgOwHcacI5Z)nj}L=iAXM&JV)T!?&RPM{SyPM1wQ2a_W{SbW2r?v zis=BlF}tlzULg3}g!l>)V zpQ&8GP+$d?o$hZkf=(E!@n&y=iYobd3R)cD{t2;iD{VZHCcs`Pu2li;q6U}-?}Kr{ zABIHzkbysFk;0taL2HDF^FUd~Mg~7EQ}BYudOX}JtG0A2g{iil`!kUe1px4Tr{ACu zL&qMu<3l{VCif$D1m4)sZk~+tSrCrQy6*W^>iA%l0R~m<%PXuXb zgZ#wtol(ZA>kk_@H$hJZy2Ar~n_t3D)z{Z=6+pLb6R`uuObB-|Sbk|g_qVFyGPKif z7v8$Jqsj}&DIE^S+dzT`wI^#Ny$8nGg^QZ`SS5#9r&ofV$@>Amm75jdrz&iL zoza}Q=znpJ8L6<0W>oQA;1EBh|L*eT#=)Xyo@0RY3=s8N*hPdQg!FjJeSevtQ5O$~ z!DhznD1sy=wIPh2t$aZoDy;bEqu_c(UY1!o=5$sgQNtt`p<)nq-|hF;#yDl~=6PTq z6q5YS#S7jc8jdmU0a3r{=9uR)Fcc9a9%7gu$q=*VS-x)mkn5t z9ZNbG+g$##pe~@Y+5=8scKo|H_Rqt-9~Ng9?sd<-#m?p~Sf%h@)pI+jOhiw8C0JWT zztpovId+@xt90_Ymr^H{HD`Ny;X8lszAqlry|7(;UtnNhf1Q8bDVxbRx0a4?txFg5 zrw+MByU5SN6=ab<(QgM_&Bt;SJkgOko_PYH&VgSD2#krj|51JY%%*QXD|DbSkIE5% zHyXR-Mr+$~DTG!4k`~{D53JO;lxf;#(*#C|O1r~Ka4($7){&7qyF)GH_hx(RQu}w8 z5yD0JIg&*4b;ZKzL!?~t`r!VHHL)-S7zPPQk3}4!U!xMxxLQNsY{A($<0$wah&_Y1 zk{z#wJO!&c2}v4=BeE}(WuPR1MdPxo3JBl2QD*bU%AlwSZv`c*$wSP{g?-gSqfl}= zkk%}2CNE*<9VI>rTn&5Jw#YP4E&90a2?ZtWs1X6#TPZ771^^Y5CyR^>n>Kx}APli- z^dE9?`}+m&@B{|78)IR56+$#-M{NJ-7L=5G%H{3Q3i^YkTK+;mV%}aTPSmr5MW;(U?Q=E+2h zVTrZfiFn{7dVbOc+3Hj^VxsVbpA+S_hgvz&%7K+bEb?skTa(|qYK7vvP@&DmR;Phe z=t2d#mVj);k}{MF7>xuV_}l1i!ZA{f~<`%g+J8M3Nn5yH@T|xoI3;_~hazhy@-x^!8c37UI#9j29u5+&d~n#jUhBQK0k39pCNdlKv(zVK80`h@EN5^-FJ9e~N zz;Xt}Ye3mkXe2%#Zat6^;o zt!UbcaTm~)qi8Lc!uEF`Sl$zXh4gP->FveS3j<~enHiE*a%BZ`q2D>91SAYVtE;ku zVUfxxJn%4Jaomot1a1L*^3D|#bKiKx8%CVHXoAXl!+! zzHgmxSa^d&oK9T%0(D^={h0;`H#n4szs{KfB68FF+iBD(yRr@sWm%4iJ$uPUGZmyY z4}wwsW11ahZr+G8oO>9fc#|u3AnasJ1T17E$}gaPmK?STTmerGep=j5#8HTMxl+CY z5G4S$Qr4h5rYB9Zi#VM1)Z=8PT;X|t=9p(7%{K4+_V}ZI6-Dn*jqP!#EAuN#7;R2X zj(>BIY0yX``Z)O9Tjg;J*aD4E?r7qVWEhbBD(#UG#XJ^cx{$q}P>aq^7Q}U>pD}jU z4DjG{50O4|K3RPUw2Ape7H@>~gVX!xU=#-~@kM2|6^^Yno_tBabNy!l*F_~U5H60a z1i99>)tn=G-mIc)Og5K1Y-)$rL-~)zpC=e9H`5v()6BIAztUZgQT&E3NaU`E%dM?& zb5ZR9H9R}$49uJ7qdP>!^6(Ptxp1R+CrBnj{1QJcrLI*X){uw;eI+4lX*=$J<|rs> zdtc_oW<9SnAKmFnr?0!wyY*)p(WJ$Sr@YkQ{Zy3hIa9lBN$HZ#U+0WXm7aYcJ|fLbDy~93Cx)^4qlP&GN3B z7ykoJkO*H>hsGT|L(@+ku!#D#a#Id@)8W>f^^|*@Li~iOVisP32&vuI)$OehniR7KSl@5Up?><{dRj$8mj3Ma?b|=MMP3+mrT+a(rG&_DAV3Odc24pm zQLgmsi7<;nbA)3CE-Jni34?%)IDF+m%@FN2_|M1(_UfZa$^XslPC68izV@1-sq4~5 zjlz+mP<)oSMSOG9KX4aKe03cuNjK(CEa7TWHUg5b^78U_oK3XvlrP>u>ft%2Rqkt} zO-(h+!B3vFS5X{niz;h3H+;s%#-^HtYQcAy!15-wh_k~M%VP+7xZm_1(vy%eMESyC zMa)DvT{IBn%U&iJ>1_v|O*!p+1Q^@wE{m$W~i#%WrPh`CKl{ zfBwnY5)1rhcTNT^LxYfCwz1;bgXi+OQB}NYNS3VNrr7z3vi(Ut+#DyV1Jq>CszEfV zWuV6AK+Sxn{OaKHg#4zVKgLcm3ZF43ef98J^~DMSbiWA8k?$xkdj*9g$2wWVnSrvp z?_OJ~LEfMGbI)I*7*W|Jvzo#Y2bHLkUm>rM3v;DxtM=G@$4f=(=_BjAcJ20dU#{88 z82RkV0J(W``ta5%96qDBA?^Sopc>Fs;$n0FpTWsTLctvJG(v@`+?r=B{-T0w_hSCf z%`ot+t?O$g3=0m(3F-K0RrIBsghK_++XC+etzISdK>_V=*6DQU1IY8E7r>&L#7Yke zOdIU~wOh|b3_(MM>yX3gfpA5o1Au_u*u_pAis(zE;3u3(@!9Lj<#z*clJ0fVjXt8( zn$l0sC%j~Fj0K)FhCa|R&vT%8S#kq3b~Qb+i*IjOc&$F|mJ2IpWwBFSQ&NcSf_Wc~{H1PrpLuwzSG`ngrFYX?JRGs>0+|hj~fl zLnFv*Oa11v4}AqXI~_{N4TTVN#9v;5o(Sk|LD#c90ER>x{5=pid>`-XK%xT)7R!;Y z%Uk8fz(9lz{j9U-sCyqC0L zMeSll;`+y*`RdmqQCaj;U{6KN&H|Bk^QHn0F9R(UPoF+T#2bJQMQ(NICejcY7vKUj z@?n@J2|^}B;*2B;*K;UI1wLiBvxVcPnYO74hZ;kP5h@L*RP6Kxb)bliXb<7FAO@&{ z+!qC*saQx}^M91;U0T07%gkBz&yoKr9vU6<(o`0a!Z>#B737=9eM0)yfX*@l=-W^VIWVa2(KBrl z*g{jFkD(rlh{+KkkUH>26<7?U)g{&EO|Z|l-qT1q)gH(V=#0QEhVeyKSjOW03JP`3 z&`l7jhPii=((iZ+E7f+wgJ+_A&xB?^sUK*|w`H|Op%Ngt-|Fg=T8R}a^JkI3&35ix z>o?Qde7E6Xop!Zs^J7lSnU=lVjA9~vGPh9V&n++aHNe0IE(y&{$-onQ)>4x>zZZsu zoq!0P9|mXx3H%jFByU{yJ7{OH{8wY_eRp~l1?Jzx>3mP_d2m7SL z|Cy%sM~Tx!CNzFPzvey6%TZk$LwbX>mO=92K+DREs=77wLUf(^Ou+keR+4*xYbaCA zT;DE;S@8vbr_U!FFc8C-qcXX&~6JwoG7ao?60`#k*p@U#E% z?+Ob1dzKD^KT+nB#+F50qwgp)kK-r2aB392FqrZ~eg8mU84x^jgXS3?9)wHH(~oan zFbYSRWAvu{F{(Ru?`xwZdiQTFGJ`i}-}m1=WPeqcnD}0LHQV@2BD1e4Vx(kIGn?0u z=#Sah`FLp3%t_+29aigQwekwwm_~=qY&`#Mx7)*US7c- zq>1p*L8Vco4VXYJCJnfbt?}ctJBrpqJhnPVnvQVH3e_3iy+`E~B9sa>E#hkYPFi}5 zJ~mh?zO4W7opoVHQx{rB~pa~O-R?21Q1Su3Hq#``n4*}`ZkcS&;^!3Ff+%35JP3FX$L+1ODT=Ci<~+SF|HM} zJR_g>OV{L0C1q<7%rl2-K3L)mU`BEI)DD#n2r z4&AcP1tdQ84ufRY)UYaK0XWI?8UFGi$1j>d5n8XM9{xbl+&3M`G*j{F);DkXZ>eIX z+00GCbo=NlsbOU*;RT%RnI{8wvrkBa{(fe&Ob2>zI{?P|_I#5Y2bF{cef*g~c-k+i zE1;qh+5!e?umv?1HDw|7Q32py6KW2h4+uVJoiqZ6Pg~~hfV=l>2L3E$6!0Mk zm3w<@5eoRZm^rW+r))m@?~Bl7drpVm;q+$XNpvmrn{Cpd-_>`wSt97&5xxVFfaoDb z3kY}qM$2)^n&p+=ysc&JugsyXr+WO13UI>(dp?i)hE8!EbkB!4-OTfB#>@gBs4R7H zROzcx&OU}#s6&NOKcQzqZI=s0C6JCi3I9`>jd17tKO>090-mc~HOTM&{xAEDiWDIXF4RyL7-xGjjPu~hsd4*ebz^I8J-U@kK3WA_itPJ0JR26T zwr}6=HW`dv{MA~DNDz^X{sqrtyAPEpQ2;KW;O{|d7*?FYM};o7Bm!v#5wfA$WQ!Vy z@6W6fdz6PTb?5<2?Syf>iCi+SR(XLqXJ9%Lo%&otux?thyst#?AI z0V4r!J-T6c48S-g*AZftKOCAKn1!ix4Om{x~4OuRG;bAojizVT*w^vN=u#L1KtxMbQ0B z0AOHs&9CA!kdA5ib`2&(J4B2HLSUsDP&Y;C_T`JiBLa4W078Zk3+2#b1!;vI*TX}( zWX&u*h`>I83=Vn;s3a+0_<-yl6S8o809o;jjM0+u4K#$EMhN23br4-3idRqc4R@xp z>hb6Dth*M_Kg0P}gdJ|3p8APHOU{AQzZU-feWm`uk?q#k|BZRO??Op5-(&rlFB)CU zDS5jb8sLJVzj5m@v38v{E4Qa8)f5)S8IYIFyp9#Lk%QU>N&>M$aDJy$>Qrbr)Jhk~ zC!z0@gXnIWd`HEo`1Wh0SITXd`&36oL}9fns+8b$`aDqGs8?R#cT;`lLs$c|h}c!6 z7O(uuX(;N_u4p9D=T=pSv--f-xMz~o9p?Ufp(B7gIr>$Ifyoqd<*Atg|4*Z@Ku9z+fsB5j8WDg-w5V{>O(QJBw*%o;4uq^O?;&vIa#&Lk$M-)M z3~Ond#Z?Byg`h#NzE(X;_|(k?)bEBmBu2LAI1sG;;s~L}*ad2qZZnONeW=|lQ=g`o zFy=8dZR)6?vwAjdF6z|(0g{g1W@i_Zx-s~!^yWlB|YeN#5$emw_NXy z6`GPTPj<|$zg;bk11XvHua3g73-=s;d4~ct;kJl5bV@FcYzWs>Redp_j(xrtQr@4! z*WX<;>U8yLU@hMZ=&F2PY_c|%cs=sCQ+LUuQYL#+A4FOz z5vh-Ly_spTI()+=1J>Kc=P%w$d-^n3_ThV^?G}37p>Xzy`Tvpi9q?4||Np8RP2Eb# z{&q7$sF0OWZX_9zk#H*6JB~e5X&9xDttgx9O%k%_u{k6T4oCLp|9TtM{r}G6(e1`L z=kxiz$LswX&*$@%C&BwG4`^_w=gVbeGLT))*osHt$OXP(F#oB{&PX#jqxS758d(pj z|JNfhF34R2<2ec&Y%Oc?)Q95fBB*=|1gh)bUU3m?X_Wlf+hhXa+NFx>gE7OkNj>@jL*3}RK zc#g_V)pt9HYC?TJjWlhK(Jsg!wFYa))IE#e46ioa zwtoBlw`7;4o=0Sm6>eL*6Zu#el4mxgmFu?M_A%Rdw$zEwzVnknXpWoHehrLgP)3<) ztXNls^a&rI%T@nn7k~|jb@?Y1L{*^FC74dy^QTqMtuC`~RBM!>To1|+8-h0zy~f=N zvXCBwQjr5z&<31m;$!iGe`Ju4j<`jXj7v%=$^aPC4Ae`f^A735ii4_m&i20TxC+~XeegzuCiWHo^vIv?ZEb5#NMMa8KdJgRMHO@NtpY?H1C@pyso z4q0#|QIZ?jwtPD)VNw%pxd+w*R4S(t)4I< zElEx$%*Xqb@g^#o2Q=8}7pxNi%W!5H^^$UeEXD9NZO zu3#%!k=z=3029G|!t9?qiDD_H}{%_r7K z`v|zQ)M_-5VGekc2!;Pyt-84k#%DS#eqhnAJ$&ujH7{p?X@3O53FG{=n3!J9kqWD! zA44PV8Hx1Q1(S34A|vp zc>xa{ihqp(%m7r00K|vwi>#t++pN|nO_5J`3jDBmV@ID2o-VV>6U82LZ9u36+_cwO z{;m_R16~BgN4G`l9cnzg zUOLH$Xv9>}nb@Brmv zym~SA-TSe&ptOV7Lx2_85qL&B7UDEV5Cd+;L$|117aJ~ZH8wx{?3X1^tgFZaW0ZnX zpL&EyO0W=6fD8c%M9y==Rg(~wh?diUUz#dvXfhauOCaivaYOBO;IKU$yj2IX1F04bK4z z#~1!T))D;tv^JzIq~pQO18#|tXe;nwyd9sw3TRE#9BO0{ga<7woOBi~w$23~U=?(B zL`lv-azH`~0}H@q>t1-Z3OuDeAe%4sOhh0a_rNoKl^&}ufS7~9bI={J(za@cjPr}* zYP}KH!NLK+hPDelXd>xZHeHxIlh&tp!fGz$W`LD344^$P$6$odfHQT5=*fuE1PJRe z&2AwfIl#@hxyLMmo`Q8p#BcO#z`*#1kWdMY#Fe-$K45*jkHj zcJJ)k@j2r%{%MpWntQFzhUqQYX)NVgiEqfs} z)U2^N0CCioL?#d?u>37BQo-j_fg`hpUk$MHfpAzsM#+n5EJDyetuwa=>Z=hu2#_K( zKO+w1jkY;(0GD9c?^Jxqw z4`_SBwj_}hIuc3qBnAVV%6Obm3h;1XMmc2CsE?^Rip5P;?^YvddPaRE)Gk0#R4vBA zX!L77s8glbLxFu(=wphru_HH-$fRwL+IHm6U-N_1RtgSVHdy+f!VRbgkY8hO%f@~F z87xwZXznXryWRvVCd2Tm*l61q)`cz0Jzg_@j82OsoZL87cW(+i_bKjOq7E~Pqpqvj z!r0MG{vDZ{29UEyYTHx|kV8bigByw?h@}s%37m7t0lP>`mh(m^OiJ0kiY;8ldEzX% z;Kd;<)lm!nA{usPThSRBB1m)q3DMw$q_{0 zdGH2gyG*g~xXUNJoizKbTV<5ma-!AL04s{oV!YDcjUr)`m9(oTfwxqr1DZkitTkxP z8$@<|c)xcw;d2}|Lc5`D?m0P4icTb-r8?c}4~`dS*0B4Bp6p&Wu8fu#7bms6aOF`U z3Ct#p40y;pJN=~S5lQoA^)*l?I5aH%ML|K#>N4ZaKfnV8Fh6GO2D}Pwv&fFU2M(O2 z{R^HN2}wZY)Cfk(%cl_1ZYYBY=?NoH!0}~T2T{9nodUpY$6pLwK3WDb=6p&b)U`Rz zWFXx%Sa^J2K3}<#=}3DE+m8wCbLdt+nA#D4fER;geV|!xDNDqz*xzNZ1~LRVabkZk z*DVzF+5+;&zyjonOOw3@jdBQji-&*|4IjP#i_c`9ibd>rl!{26&Q*rK{ z4;tbE#f5p6=VuelXS4Q0iT7hhswvi#smMm6+G?XdvtdQJ9s+$m{0^&%O*1Ndf>a&t zd>P6XF6>wZxIAqI;l~hSm}Uwi2#mC+))og?ntDL?hRQ;l=*x@1Md+j?u$WynNt!$L zKd+CU!?s9#WCmouH9<)rQ6nn+QFCmBjS)P=bNM&<-nsW57saZ%tDSS(?k*#~Slh2cA5}@&|wq3^8G|9mD3zZAo{um1zQ&Ibxc(K^(4!p^Uus zE2T(w@P8m81?i~PE;>9?pu3nC1h8C0_0;kArT#it3wL$fGovxqIMHDseSfDvF%f48 z8V7l+Y^qpNAKoAn+%V;;Z&|cVZBniapKdZSC2E;+&-<7}Ri5hn_8o80ue&ke4Qc9J z$B}f~sjnfgL)ak9fsDy0O3>f|w37)0ZDHSHIIX=05fFHMziSTMt#ou%VWcn%;^TIny@_w2EoD* z%3W35!55?ep4RGFABJp+DwtzPu8wLc3jWnaI(MLI%c8n22$W)NH!AU;>m;&}g-TJk9%+cOfFFq%BEhH0|VsC2PkBlV9mEW_Clf$ zM3Xxl;}fTW?acR&Wg=3(0bhV_Ds9(6pY5=Zj7+Wumcrc|s)aKVJFa8aG1%m z89izgx88TKuTxC5@EoUNF>SM|9~E_fO6EL;(c1Wqefx(Jc54Pt6Y7bd-FFGQtmJG) zP-OA1Adk2KYqu?SP3~L;g72zJ<%@EnBPo11)fq(9Ht-WFK9c~Y<{$kyxil;3OG^ZN zBM0mNC0t1lgp5dDa#GlFzE?JQ9wBfC(YUnPz~X{riT_QShdvB{_eiMG8_;)`ZRR2Y z?7$4NyCcc{ru=8BI9|9;kA5?roL7F}L{~W9a7^Po`^!6LN1?}RDIzp!%o4;yw4Qqo zRQPV(KI-rG0Zpky`knohQ+ulQ?wjEdXn$H3(+J!#s9~J}Lje;Zo05DhCT_m=Lf*=? zCd5)Gm!%hN8vdsM;$)P}ftdhz-wbJe0iRsYtKY>;1~k$1HXa(zO4;}^?v<4$HN<-b zydo#Kz|mET(=ac_xW#rY$gQ37LbSFhd3ac)`H9#TbIzcCs#h#0;hm^Qjc#+}6`b-p ztCln80AGfR>smSrWdDP(A`5;D?iO?9@LL4Bhrj|nF7?54C;^Nx9I)Ou$n@e|j`HiG zP=?+RWNJMze-ElApe1JuP|gvg9$2Y=5I!^8-Rw)88R3;;>xaM+I%@)r$*`A4EFW;? zdaAeCdORAHe&eyW?vN3H2TeSzz3)nea+iRP3-tai#x_(#H3rooo8=EW<%s9egtx~> z4Ol>$;fUFZcR-I4^TT_lW(Ea`I1q3JSSA%c)eoFyJ2mV=1UCl@h4kS-*`wMh)oz?H>NJ*o zTiqZ`crOLO5KC^&gQV0v)VFWku(b5Xw(70qweD!4-p);JUtW<*El@kX!YG#3=oCl` zs_xK>6#A`b1n;VCSdnJ(XGTS&FQuim+#4cg4G;euZa2N*>H7Jrpf@LKydgdS(s$+f zZLm9hhs7eXJvAyhcZe1{N_B7ZEg;b?D!gZV|3HHFv8ShTWqN&()|0@9uj~JYUAZ!ksF{Hr5EU~T0UYuS`)+zRe-)~ z88V3%jVTy$86|Ks>6rhxw%zOS4_kP{Vu$dHFP7N4RIq| zMOY0X=Kluu_4P{$lgOxmm4{3KU)4e~&$EpZcmM>&wA|%43MWs9F9nuH1#vIe0HCh4RPy5W*hufei(oMwYp`$Xqx$YzI^((Pq(r0 zWMo%S3f|?ykd!YL=ywT7$paC$QoSH&h~$Y8=>w8dbRPn0xVmVu$sq1@Hed@<BKp6{2)B?8LE>;fAqxDco;1b}n|?_31%XP{zX@>7`J z=g29qZmP?dO=g*(K-hQwI1IGqK)I|R_@nsK>y&%Q$P_V9y=hH!-q5pd6g{zxK!^KK zg*3uO#CSc#@Gm35{O=1}eB1Z-w1n28SK?F<7Ue^)nKvb;310s^O7@_|GY#S)E`!OFxw9}=Wx{yUnUE9G= zpOU>>OK9M!?=>qaW5*WG0WHrLgr1v$p>+)oGD0y0>{-4{ID-+T2o*g2W@dP0hZzrS z$1@-W832%oQ_P6EhImmA<6(mWX0C;!;ufS?pm;t4f?zt2u0b&ap-@s*HG!;!+tUi{ zzF5(AvU6VyQGXL+aK9dq}u| z432{6vfnv9i1QJg#W*5L+Lvwsc!*M{TEs{50&)T`7~3B1_8plPbp&wr5hP=wS9=u) z$opepxu>NS!7O;Qz(XOuF(^True=o%Z2yR_36T~&ff_SznJ+~)I6zqNHZKlCpYiee zf`{?w(y5_PVTDZfYv#haC5KZjA+XMYDFBM0f-*{*Y=UM4D2W_LJ`3fSl1t*`&b;Ei zdR|8%hlG$NkVU@egW{4eY~hU?${twi8rkXi(#+%wur7NXDwqebP|)%&Tx3Pl8g|h? z#XCd{32p)fnFC0Mh0CqRe$fB|@zMdQumNl#tRCObmXM-}*wNn;F~J+qQ}LHuH7-T3 zmq*5-uVh`pd1>YxC}!iV9zV)9?fU4?sS0_7`K2AIaMl8k!PF)WZGxe^d8<^J*KC|Z ziSA7A<3j+5+Ts_J6qn%4Qfi)L49?5Cq}MkmCXCTYjRHA0cbK-7HM= z$n!TvpCQAC@(#Y^*}lbB5Iz^&SlIT)pz#7q@mw1&@CB%aqWZ%*{Xnz~re8&0&ewu< zSTIxsC`TRf`MPgp7P){@jb)C@$bdc~WGf+M_ikLi{7}k=J`P&QEdi=F({@7p*fABG zagdZxm|{nr4i1`j=S=2-;h!b8DFnFUnRaUau`Hxlp+B3tjp&!gK@dO#9=0DdLQ)dj zAx|KsuIdLy9J??7_%Uj_fptE^jKh>R5N#`>q|TixP~9qcm=*X19ZIYKyVvf&BLn;~ z4MErBM%@M2Y>4pY7riB1jUE}!yq59N1EFuvDnYgf%C(~^*^Ytzz*vl%cWRh{@$M+*?2~+HVQu zTLkSnn6&Pj2IKAa1oUaQcAc9r*m&d22*AHUb`Djqb)3OtU0^VSZ6~_yRL{;tHbkevOZF;=r61FO8pD zEi~E>6T4Y@M&%nX?rOl?vB;df&{ZtLyWTF>af~~_W$Zk%7^*MI-q>$h7OvfY%)qz8 z5VHwwQPHfzJiV|Dn2`@8Kp|>ItangB|DOyn$q6i;sQ$IZCj_!grlsUP9Gojr=@WoT z7Bw>1gB`gx7ECBZE(a=b^tfyFcpL>~m*p*?02s-uFf^ih2VhZJdjJl16Ju!lNJ~XYkYQ)AB`N zHna|p-EN+>6!apYKa?wEYg^SyF-h zzz*eGDA*hlx9UHT@J4;0Zv)854Zd+6LUu!bd(0M#KdrO)B8769-Wb>jj427kx#jFy zCi?=LiE>A%nAGc_tif;C>%&ocf#2`0Mfn^fi*nPm_T=x~cD zL{S*=nOI2FwV>M1Qn9{MBJhFi#`pDKYuYL$!$3U{yapPvcjzJ01Pb&Mv=yB%p3j21 z8@d7nsouhHN$|g=m~u*LXESbw4PX`Mo~4W7yCPU_7L6!O7t`ho2L%>IP1CB9@PF3h`BDtsx5Cw}m z`pMD4w(|R96!ZG=0m~<|3=bl@%5remqr~iAfbk!CRDz`mx8qm`z^&8xF%zhonVQ#?y9J9yM{;BU_-|F2uLykAfCeT&Gap=$71PnAmhuo;4{N&g{%QDO6jzJe6ZK55Ycv z>(-!s7WnP$@*M$(qn?y=kOhMCPWeLbTBFohi1L#>O32*u7I8JpEHZ~KF098XM2MmI z1zrZ1ejh3j0rOhV4BwDZLXg^xBomp&3D@*3BMk5UJeUF?4%!Q28ndIg>M^B@)}vJK zjgxbwKwoI1#Sgs4ja+hDMF;-Hmj`H%)w$G)8t&5SRbE%!4Kxw_;-Js(2o3&;UY~Uz z{m%7m{)o(i=69la>qpR(@Xy>5bVj{k&~WDup$@W!FTX=`xFNmaXKWFhWC&;vHTZ`O zCv&^y!z7*z81>>2a?V0X$*@9N6^{ZM~yzCf6-|{~&0;wyD;i5}U^^8~k{nwyQ((w7yr)Tpo z@*VweQp8A6Q!|l8HoTGGICs?#5Ys3IL5N(?2MhFMJ#RkA>h(!<3AicodWMp^t95~PbXScGVA_dY)O({szl=M8P zY+w-E@uB43wLcP^Ff|RDfJc{$nCOrcL@}Nt!UjXilzHSCUKElPXK8-BQ}Av#jzrDW z%YFc@;OM^tA09)kahTb8cogjILA`j_(ccaayqoN2e_x3Q2S*yBrvY$oWAiFTApwu8 zj65<+#p}y0V-N29^XFn|IB<}bPu+w3pz5ZAjX|c^x|1(6swD!idxP2u|mYHXxzBEEaUh5`54FP(~4*;SW5o- zoppc~1DOsuj#biR1zMFxL6I;5vxFqdnMWGlNy4N2k(z=j;snmC z$V5Sug4iO^;4adO*jyK}b@>S0hY=nVlfm z#-3MXgSbgA&`s~h*ex*>-9e-~^-G(t;XLfctA5ZMjz6CY7LTwHU_@n_4hndIiZS0V z*>LC0tHP&f>9^LIg^cEoeuy-ok_A?#hkL^|5)) zG{Uxdj2xsDFe_j-B(g({+Dvb?ZY;MJG54PMtw zBTDkC$}w$MbLKS&V8%cWtEEF1aU7xWM;Fs01kX8 zG0tO223jF;N4Y zD$?&k10f4XZqgfw84=$IIy>BiR!-h^CB=^4_k!&Y(qasLMOeqLD|bp-&7`=lZtfQN zYFe*t@9y(pKXvijXi7Dk2F{c-s4fib&^OEox)T4i7b#1C4!xO85XzI-_B8}ni8!E? z#Kl%$Ist9Sqv@u?&QHZHfm{;wiGN4x$wT@jdVx9%5UtLXQDb5^GY}h3^ow2BaL0^6 zKwd_mCmPOqgTd#=kR^h#fu*GvNDc?qyQfe0tj@#;82>mZ(7>NV^R2qN`T6?d>n8{E zvu^F)%6oE4Z_T@e2jr9$A-{3t$PsA7fdu{;gss1gr8Qh%{aJd*VbQnRtIdk>8ZWF_ zIk#eGzAVnJ>^3*|UB~|Z`_6Vfz>R%tx4KCkg)aAzk^g)A6aq#6F!=`7ke3cV zH{kNy?O3Vh-g^39e@T>XTc2M)`oA949JmS_w7ty|jYRrS?A=_k#rW%5B1^<4#%IzN zmV>m8G?@HxNLR5F8mFu6(1AP$C{Hb9u%*f{(*9botOjtdDocXS2Y zb=8tVZ88ism+P_c26VT0di7tyyZ=*nLSmY}7cZ(MtgYy8FAzm|ryPB2IU%dfR-PeCZ3by7YV-I{U^ft}vbOwoq z*#^wDnaPs#Le+csMPaw?=EhrV8@p%8^4%;ce4QZ$@i!AEZ~4 z2X9+BvbV{|R2(6D3pED+jO~#y<)MdO-pzoEtcYzN zCp4J54lR*dPJLBa6MI+&`X|JakS;@56|GJW&``Bal1S%GSKnVh>Y3U2>GVDSoQpqY zHf|T)^5#o!2v))BxdGJ2&n{71<>tHY=0RF2vm2hDNAnl;7%WEbm5*{Z1ES&gk zvC)^=b-CdB1c^JrrXZctaIkg%(jD_4=m;2_Uwdy(cjxHiKV}2gBU(ZmsxX_u7~fN* zqOlx7{l>oG)}R>P|7rxI4F}MI-#gx9Y7!;Bgk`{| z_|E&rDK%yMn1ku^?p5nZzE6S0^^|22vJ$^?xNsJ?V(<>KyYW=RJ|d)>$dw%=W8Bbv zLBX}bSrMx$)L$5c^9#(K?(3e-`I*0Ox7H1ssU&%lz~EBeC(T`@T%a&8VHHuwurjOu zKq6^$p}*jbW1c4A8V6s2Z<+qj69%MhxAHSzg)dWEJSds~(yggb`QuwHN;lRocrmx3 zAAn3j0b`Fv>FLcmZP-&6%9)}<(3U|=d?o2j6;FwJ2~swY|8Myyf6!H7x~;*gyD{Q1 zPv%qOokj5PdI1}Aa8x%g>-FR@Q_l7IncRJ*y=otCtR^rXGD!Y?8~XL(xVo?iSdDJ1 z<#@4;xrNYv=JJ)ylbW{Bc^3LP8r{h$Bh1M&tYD}AC(f|=QPa^kP)(%umS=`NCJ5j{ zoF6@l>q*61Ko;yO4*EOdL<+_j$+@{+XEAa#*LoyQ&7WfS3MegJliXTA@E1-5bbnAd zI?d#aY8~k1Z36wZhs;mMsR-HulN`EvM^;&Uf(D)u?=N$qZZV>gMwFvNpgd8#Tm@pl z@w>`Edt!FjbezxhW{6P*NcsRXJ$KjnhR|OiYxr8Zk5IbZ@UCGpnf2@ZiM}+w4ED~s zFraHz|I%SeirHc@@C2xR^((FIft~ozz&ZfK4q99rs=rYIfrc3nUNBHgo=}_yD$MMj z$*31+HU20!ZafWHpC7_ZcdZvFCis-VXg(|LG*Ew&S$?j1t!Jf8?Nj!)0zx=P8)`xN)}24e>|h1zl-SrVC@FYJvQXgXhMsckedV4DK%eT}M5#*p%|yoLWd@;|BR zKNmDv@H^x{zGjvSzIf~1yLX#Oo`nqFnCnDa?FOX##JUOhi`5E=m&^bO;&HLjmS z^8W@EX=Rg~GNPcXxcKaZ*^LzWwazdC^kl`^a)U^cCWbn|_)ZRV{dBVBCLJ|u437-< zsN)7~{wCHfnsP^zR%zFTGF=LzfB)QLs+;_5m8zPRtA2b#G2$#pe#T9Y@Y|HM?bTT@{TUMb+;p{>K|GOD6O0rJMeII2?u1MlBZ2*hdk^ ztoGk0*--^Ch$zu78s5LzoXE%;k4$2ki0REOc&m!-LGHD8mv2c|8I4^on77NFf_B-~ z88}YnGctEeHl-3SswBe^_y2F1!~mv>^o0b~HT$o(65ROikZ)V?vvLqkZ8)R4oBj&A z0qA(SX@efqdKCEafw=iPeWTQP=S_wK+*W`Uz7Zo&)yjRJxNmxCA4NViVvlpRec^ze zY9n0)MrVS{7>HnChct2T2I7I|2Da=mdC`~uIDl?aZK*oa$Psp!9Yrn|yxfF^t&%le zYXmU~X1NGLejz@|;nB^WC)t}5GrX=p(L3-yep=k8_^#srlus16KmVuiqnn*frP9$? zi5b_G(=+&$1vvB9s*G^l0l7h$J*`!$TItrt>9{jp=OtyhlnXFjKi&s?_om69ZKPRP z?SF9ZElu(lw}NnAY2UhA-Qb&<=^1TZ`V8e8wj_35t@v#Y= z)$7R>2M^BSuwhOa(~3pseFh<{%1sMF3xAGdw9KjG z*qA5o;T|P6((YM|7Q->2O>M8E=eV43$+;Jl5sA1uaoJMYQ@;AMr(Gw(23{#tMDcv&1nRe-X z*E+TGa}SGh1pkaC!uYXORHpbMIi4D)8>y+Oz1$QQUyM;|rFRHZUolMC9sMqPNH;`H zDG+1Znf()|dzVyN;5TEe?S5g5=XOJmVtK05YUt~UbJk<}Q#WE-S+5ApGZe&adaP%pIm zgUupW3&&UKb6j#peG>zgg-kv_jPd=?v5U3oviNb^Si@2$tS=wUUBY|Xh2>a`iRfS$ zr)bR>`{x5K|89GQef*H;14)($TK;`gu~t>bNnWskG}KU$a%%q1vx*_xj{{#K;V)u6{POt90FzNq zUMA@TZ+X|?#$=YX>O=_Pxo=!@XC!9JWzD%!OR_n-1Wi-dB_^;gVtIm>1M26RM-H?q%_cV9?~ z)3cwpybd4BJTUnx#g}05v)nrWg03JHt>#otykA;Ha%q6ms=cB_CKFxkFC_qUGy1b~ z*OZ)M%p>h_I|n;JL9Lp` zaMhPG({L=x=5Emx6p12#9+IZlSoe@D4qIK#%>bVyCuh3|;@0-a)00>k6FvpGgm{`? z3r)5+cjvf>U&^W0otftS39Re;$*7sq2VP2w4^kOi6~Oo!GYH;_Je5nCd1zW?e|idc z^wF5@kTSha|Ly}J9t0SnSf2u3lo zMrX0p@-$x848Cf~=B(kYwrBIS{&*9tx+=l&l;p##T1v$adLGqx?0=J1@s25dx`$N! zPgS(L)>Gy0-@kBfN$SA&t8ibqpc>(l^<~SVbANi@-}ikA3|^T{{?r(0KE%n{n&>}^ z<7s^-lu$p?AVYZUB6P|*z5d+=Z|4^78a0=q!TtLP1nS|6C_+AY{!!W=QEZ+I3C{ap zXY-u0DEVV6Z9EX!VAg76sScD;T?G9m&e3b$Y?fk`E3Ax`qp>M0UJI=Ywd!^4yYKGv zL%2n(MvbecNH>1`>&SFT-|ONWo}cb(v(}Pr4(I)Otq-&u-!=i8-wc~I7Y^>zT^fqt z-(u>h-dJW%=SeRiz*@Su({ysZo+K!+L88ggqg7>USZ45+yV3H|Gf^Ur;nU`HcO>gk0rUk-mQ_|_HH0% z3j5u^%WU%up6;E$V*3KUc}Q+fK5-8g z)_o3_Q}ni**f3jUk+Bh6yMw20go|}ssQNe&d`fywt$EJ(}7K`*=iMUleQ8(!{bJ4Ab#z#q8&E?w}D}=%u;k!gbpDTx7u1^lvB% zS+<9KdJHo*dt(PzR-H7%MRZ@U&EamL zJug0&NTB*@%6PvCBlVY2En|LwnOF=jL9?v@M!BcAT4DBjUCAA37xw`3)aG+1u;jl% ztYnIV@}Yw7>eZk*W=}pm5S=jl{2lHgFn>YgunU#XwJuv-s)0P#@}R+PV)s4d?8dw8dw7V0>YSmWkt3+ zhjns=Z^T(2tCOQy9d!o8xv;#MCUT?B=+xi~rb^#uyYmWd$U|-isiF13S(T^PD}Kc+*1X4$_^!noY^1dGB-^ z$zs@BfM8VFG{{(S0De4=7wI*Xn1iKxkWNU|?@efnbn{0sy_Pp=dP%Z<*|*#39P~zY zz4e0>W63}m22^ncFtA%0(D&T@4j9;PvrM~_uX$aAR5-T_jvtUq545O z?fAzp#9D&QTk|!m|4zSEuMm>BkaHrNKr-xMc|i!YTiszR{3L|WS=}wbR4nGrJ@;Pa z3x=)F(?=N%o1Sux#@+3kaPK{?_gdp^7-qs}WoJ^yK!ewyy_v7BSznaC*_@iDU5B`0 zLoT*bN7BRjhwT2lE&nU>$Dnf>z2XB(&D7Qt^5fQVJ#VIP)Zo#PDeky+E2x*dBn+ zEUGitIHlLlIQhYu;~(_zkCaHW8TWU0u(ZzB%uxI5q#CrpyfoUDogI9^a7;Q!8 zJ&QMO;WRHWr6)cKkv76T_dRyvL|-b2TH8U(@OD-lYy;YJCZ9T`GFm5(vhkaj{PV%I zL(j``1lLi{Z_t_XEZ|IdN=>3iFyA7LPK{sLwD zk)vH2jXSxc^Q~E%G+q73n_Bx`3c9+H>2bnLL*|+h%>%Kw#=mR>$zwt5l+)}tGS|oC z`Q-M+c=JGJt;6G*()W2W*aP$aDZWw4~xpE`->abtT$ct$nQVTD>-AYZ|ocONTV2F)omEW+aUz?MGq6xbLw;1t~5CX zh4A~2TwkHuXX!$X9;n$qP+}5YBuIN@!Uc($f#Gf0bQfrL^v!sk zVW<>~|I{!v#T(S!I!qFZqLQ{-qC&3x~It#Vw#sTB*h6)#tV6(6jh&S2rhyrx*Bt8bg1zM*a!PQx@7-42XDM9rnB zSMHLnlc@Tl*J+$(U@|hqGeO0Uj?2Y1Jj+~88GacgNt>X6H|Cr2vE6YdPCj#?M&>nl+5|oqO~U_jHO9IBg?b(^Xu>UF|O<~ zqYvyKKIuK5_SGUIfBHe&xsJQpoY+|o=Ntof=-;J8yl~j4@w}$#si?5g#UBHQG9B9} zqsV2JEE;#3F4!b78%{OXi!eA=f1Jqd@5+?tw;gM_62>!;Ov&)Qo7|0;caF{-Um|JF zH5p8mZLvnr6)G+b-^JfoesWa&&Px!6sGfD$;g)v~96G<~moLkExp%|j&TMHK%s;Se>knL-+kAfP``KFg4BuT*wbv4xM<*X&smEpBE>#Ve>S}= zS9YSPbg?sHT#Wu=bVx)wORL%)=QhgEFIpp-^Ns6s!`k993G>4~9ySbF$72;Jz6Zv3 z*gf7l!LFz$9sl4`sPqCK!XsNJ6SxXv;@`dQkZol<-u)5wr~Lscwz+)ObC#2Sb++%g zXdS%0U$j{%h>@(X!~40G7j}I}Ub|*>+^LW12)A%{g-T)O3Z6+wRPFTgx4WWt^YB&jo3Ze75#pxNrSfO$)6E$q;-zkJ(L7VKwp;oOxaEJ$|^9+>xG|i8=~g30c2~9uEO(`#)eHIkyZm zo{?Ey($@MuxiphkwPz2t&5zmp-{f{(BoR-vm5fYI4zN`*WN1y-I_NCV-S!MTan`Jv zwY$A8Ro$zunDppvE#rKUhX>WY3v_K~M3&W`X}xiOZg8THzFV$iKvCnPWa-h-k*O@! zASYKm{`^-pWwWWaq3fPQAHq2;d%o$3Elo#~UAZug?Z_s+l&oJ7Q3$|U>JCvd)hmeRGlV|IkqE7+qrY%@NR2qS#z0)_{7p};yrRH>O|xNx;b(sAc3lV@UD zSzmP`?Y_pI%S%&>I3iMA=VBTVJssvOf*l#6dY%bZ8}m`Fu|}=eS_Gg+YTb;t{leF# z*M-JI7W$a^O{jkG5!P9K+KK~Qb^m&4dLaa6>(jtbH0{ulg~KTc*@Ytupopdl%cUwd z#$vUrVnrr(eg94Bu;vTrHB39DgpwvFJtstP!gs%y`b9fC+R$9;h53!upC~G?KiglW z4L(F*mwaPD=xeFI?CQb2O9SosUZ2hH+=(PP|Gd#--}di#ch;@xp_N+p`vu_2X2Ks` zO2b4tibKj%nY)hBwq7PvY!h8^U5}xlKU8(IPTml9xEphaiGJvpLeo zX7TCPyx2l4t5+^Ame8y#Yuz<&JNoh6DeLBj!}8NoUbIos#>u>lT_zV|8m3;?*YXMf z!=_~wrjfNz;9=4FqDc^%P(p;&&5Xkft7cm9VwxGBNAix@oD7Ke$4ArD1DTliXV zKX5P(66~&Hw-nZ9T@DQ`f#6warp72E;{l1%MDpzufA_)xEUWdngiN=2%zkk$ZOc&M3|VGrwN zkTFxfLzApC*B=qaC_evVAUM5mm(KW?mzIWbD*9X3Vz1>534 z5EyxUct@wp=Xc4bTI<~B2GolfmkB6EVbMX|Q6ufjNEP`_k0e>x(duMc`QUVYvJyWe zdZnk{skfZj&QkU9qjI#b_*yKi2i#kUI&Tk8!{Mi0JvO6U&xE2=Nv=QkVu$}$duL$* zhl5tQabweBFx4SuYL$!hXJm3!Uo;N{)r)I=U;9E?b|ruI+Jn8<1Pq%Jy9or6UDB5> zK~XchV@Z_?f#QEPU*OOfdRZ=&tqTWa&-r0mZ0^NN3?kSYgzEgjWx^3KLjngFKPudKoJq%?i`KE5=h9v{gF<23 z=)*yzn8eR#-rCX?c*&2?tmbs4QRDlX0g-a7g#-P)3!D}A*r%N$^HAo1U0~7J>Z2rB zqujBGKnHou*e7Gnh$SPMDaY70NH!gz%nIXbsE!~QVpEg9J|+<{7>~Rqp~v3)p`g&< z1KHs>%8_Zqt*z0J;iir9F) zIOTAnaxhy&NE{*Dse8{}YDRP7kT=@jlUBK4#bj6Pg*QXViCWi~ZU(m`)fZy7v;K|X zzWx0sh33Za?H?F!j1qR7JJXL8>ohIONi(e3qLO8BSz_^V zQ-e7Gq1fwJ2}x*T#;9D8Gb1|vc_7#P>8dQxn( zshnB-wL(z&F5Gs_V-Qgx#yc}*J36GTM?M@3o6~X_`7qj+647O^Ul-aK6X6mMxU2jg ze2%3KqC*T=iZ2bHN~zJxHVcyvw`Q^yahywha)?QL+=>M;F7{DU+T@$BuQ6ph&WCdZ z(J?S2wYRHhFHlLqJE{2;Ulj<&YllXyc8lH ztj1sMGB1uh^YG$drY~%8{@u3h+#H|f+U_T?R16Av79POI5v01l4Ra7<=^zboe8nnA zZ0)CWlGb!)wdFonZZ3@{oFlAn^fP6Fhh2Y#}BfS7lM(Nq5MBSl{V46ks!P zBWO9G)2@CX3I19N=B{@JD;wI{E9!{-@$D{vPOsMn^EZpQMsO-0b16IDZPh*wIxcUS z;|!bijxDJ|V%IJBj8L5KVUyKyf!nzR0n?`TcKP~m9N2=Me){WYddqV17LMg-Cx6S` z#jgTo4+asHz9#qVyOF--Y^`edC_IOvLveBUc$Vd8DB=7!(2Q`%D@c-I#W3m8SGQpT zB5stQYpW|BA{Pth`&5+t)Mgzo!=^wWE?av}RsB?p@ZR~s%ygB?dT+^{7L%1ESl95< z{b3=QxvkR zMujY8%W|kBvhVv=l8L0TBs(FyL-u_uLbC6UeLs%<*q8tFqM7-=|97sdX)Zb2@AtmX z`#kr3KljsF#X_fza)U#MAeH!;{1LM-M?KiIgc@QE%vJY))j8}TeXisakdQ6dFyD`G zee86MCAQ!6koU#FSqV=4?o&+s)Jve;#0t=RK6 zLx_>yyq2jZ{Q*Keqi^?q1WY*4)x`eR`zl`6XZ5`-q0E6f+H>wL022Y$qdW=g=DMDI zr(4F7BL`P=)b>-Mma|=QlpA4Vw{}m#wR-CUQT$|ZZx}>!lwgj9I(GiZ7pr_{>`Wi{ zK9Jk*tb5zf7~y~AU1xo^XG7XtWovef+7NK#zRCdCTNy*ex|k?0S^ zq3LY5l!EDu^`$9u`yRf}j@Oe0nS6X$#73#9KX)3|WT|Ng9?gEnFv2)2vXP;XW>=re zG?ExCzy9Ic#84kg#I*s9da=H34~70ET*laiRBhcrE2?eBZzFl-7KaZVRh-@arP7Q^ zh!g(7s4sYw+7@&4e98QvWgMAR)EDP={f4@V)~K7I-KHk6Ht9tkzpGt1XBC2xk;F}C zISXJ1dx|9`CtKI`ao3->ET}UnQUger(NZ} z00>#3;kU@=!8s5AW_>on7Gba${~HExb&f-a1inKTePfh366^>6CT9&hwsMMiqtNkd zp#93$VP^3D{apWsXr_HjsaC+aGykTwN5x2CrT1x_Utf-KJ&5Uk)i=eRW|%&zznJ^l z)3E#2eq6yVNhgPbB3arlKvn@I)8uMD%&VSpYvfpc_-9E@n=YEp?<(nBP9wT&PLzH) zALDV5^nR4Fw`f(}G&G}J$fqE`&9~4PL4JV!a}ch!_#d|4qvoU1Up|8sGV?f<`8<@J zeb$0_4|Rv z!GipP`C(PYo=O5aFvYOq*19ctw0EkZMqRofX=0vrzo}JCO$sb z3aR@9MNs=#1cpli=085nfRJ_rYNe>}i<@ipy0xK#9q*r%e^BQoUUE@t{#3UJ*w(<+ zdd8*)Dcyhi6EsYUbz++7<+fM1ucF`3dq5HENOIS-HBL;&89mAD4P7sWGEHjQrAWba ziCD6!bWDt|^#i}|5+_amlFHKLSwJ<;UfXmzS;g@C!=%8-eAs`rcx9|)x2DI>271sf59&KyaHO60UT0zBGyBD_ zx%O&~@bV$$B#CW&fx1kHd!kjyAs-gazlSq>%aV^u@`ws6XgnI*0wzG8>9+|pjmC6$ zPnX;$Jr}y=i{lgF>9(v4g?CHSb2fDQ5Jket9v@ZtmU+TFd6fb@Dp)772l%zp7T9~MbM^T9 z;WZqVwjx)qjU9VgS@o428ssNVtNO;Ug*(dD-*vh>`fx03ipnEAXzL>|2e6Gi^_tZ- z{Z1wgkKGeKy0O$_!)r9Dtz>?&h3RzFJjp_~7|1B<-VTAg-7!(PaQYLG-V@e<_o@sE z0#?h(UCHW)Gl=A0&iN%)r^~8F*lF3s+cLsCGr_-fhIWAvrI5!KIx8`b&kz(fm0!dQ7%mtlZ z=e7TGChoXV6ksHrHhSl-CcB?l#~QrUl~!{ZO)@!6u$gkqYc5U+;kOS}=`|1lXhybd zxEy!=lJOMwWEiteo^V-0?L0|Rgf(n?ZLuouZZjZoI!~tp&yB^T%1uVW{i4f)8Y&mT ze55ISd%{g~)l2~Rp;)!l!HJGgQST?yfB){?aml8w%~+S?-yVUjNiSAyYsRO?I_i?7 zov7s#pn?E`ShMUr5!hFR#mF^e?LR1Bb-}v5(|K~}n51tlKMTN^n>t<8IenUEs);#F zjAG8;C+J+lL{v&P=e?@54FMB@l@Qv!{%YcimQr~@YLtCa>&LG%KNKb>aW$4dBvk*w z5M=ZAmjE_-HB~}lh9a@an?H2>ay>}X6b%iNb87eLzl7f8(~kz=BU=%Au_w>>5a-l? zCDU;9Jm`qnxOeK)^7kK`xaa6cjc^uU{w32FU_L_7LwDQ)wLFCIGGUBxJ`xd@TgSmAaqsZ`O6ON z6BDuXz2y_37rGA2Ux0sydO8!vk2R$)t#Oxd!n@4aD#dAZ^XERTR)z6y<``&(l=^yAk+)Eo19B9c_X=u9m4e=O#NNs9|hO* znvYHtTG&&gpr)&T-W%q@sFpiz7*t+~KbutS$1hL)Q*9Ob=E(k%5#Y4qUvfI+QC3g~ z12*VXfKh_0X?ghTQD*xjzF54q7=#nF>IU=$v8U)uTHAx>oo9IT1?-Nd%c-z zb!3B@k|*Fc1l2lK`RvpN$klPTeVCyfYIEJUU-&ILE|ZEi2KdrA!?yds0l}2#`0dkW z`HkKU8{vcIJV(~KV27W?)pJ&M0%1u-p8M7?JnMYV;G{{G5yP#8`z0(4lYSdV2GLXf^ zyy67tnI~ZAgUh2ro5B<~D+5-a?jvjEWMjL{S+a2-{`jU{miKrG#H8-FW^&`ud3vll zMMbfM`-eJv=dLG#R0Y0^vzsTWdlTI??S_mb$11WV_LoW21<$=noow8E5nd5=MVEC` zGcclhm|6`%&S^ya(9Gq^TVPR8J-BM`IGG_;|BdH|Q}z9JZNQ@K2lCj$@;M6@x=nQM z@VX$N4H2k0m#6ilLXFmx0?n_f|frtBSGg8 zSDa4jNwpe(QryJDq$6k?EuKeto)+n9devQi7*##AA;{cG#Ep;s`tn}UPaw0@020cC zIhTBCZwCG)YYxr!w0rNKez(U$M{g59N0Q|k&YOD(3d8jeA3d$hjZV=|nqh$!Kv4}c zpyb8vtw?-<^|<)ce0#Pfh5BA65oaYkkZ1%i_#DgB?tY3EB9s$fE58yvlk7Pm(rpPk zq`uYBEnpO5@0e0lzpGJL+l5Z^l^=lyRoQj7L;#ddfOG5|)Dq_yFNVP^f(_HWrKzrg zMaPEi7j@vyw#laBM#cQ?KJZ|OghG^9L~(n(`i)~iWE%#Rv@t6A0F)CPbKUKr{miBW zre0{u+Nd}4Izp+9sX zT^H+sH-Hl)3G^{G4q^u*pZP-SyeZ!~Eh{$sA(5C#hMI z8l!Ld)4gTl8@^>V(Q+?=P~7_5P3&}88wff8j7icqBqtFZvbtT@RVe)9P zK^oh?r*{j_m#Tzl6H5!fk0BvfMH@F~HnbLwWIie>y36GJQ_sD`Xlgx7J1*8uT(e9n zT0C$L-xVF6YHZ?L-5z&$=s87oTx4XV;pXRk$G}1*`q`DI6Mx@kJS@WDswy5jF3aNXo}Mh4Djwe)wQX5(UB&T^Jn#1ug%wSpS_tG8iOE()>NXzkHQI z;-p(=vLWNsbRw|G{yqr7Ow#+siWnIv>9~sLx6mxHsrM- zsYIgU%-}iBC-Vyy4qU*PDe?F5+?NUAyOMgdvFqIFeWN#g#2j&r_#8djfx)qj(Vpe7 zRALCODCT`ic`Iz!W`Tks%&dBR%sLmm-b`EUWQ7Dp&%tsPXTS3%xge4AjX`1p|bCZ115ZX6hb1Q?LdS{b7`naQOZo2|LRImA5af^b zzJ`NiX~*=DtnfQz@MpKLDLbu^F4^2(`~0h?mp=8PgGW9rdU}iQhefmVhS7xWqGO_J zQ5M&1&(u!L(WsED?HS=HgyNCDObw7=@b~y`MAr^4Wz);s7`tu;iAU&m7N}%V} zYcD|^zkgNJ!Uyer9#GU!Q(>h0SH4U*7UgV(E-_)c?V}x9RjN$s8%x*sBDMUN#^~Dt zetyv=C7a3>CZ~4?F}Yq8U_*4g#OA3IhY}%+={HhnM#a$YS&JM2aDIOfLeV0I4KDq{`m& zm9@jtwECccme9j@sZ*_S8e^eOE4rH-%~3q=ZeCFR3p!s^@6NH0)4uGQDDi-f%r()% zz^BHlmV_INp8ng}(=Zr%9++w}0)@B!ReSHswk=Be3)6AIycYy4r=A)A4idzz41B6w z&inh{w|jug0~RPr{tkWokv$%P{c(FL);H!LB}U28G85840N~_rCa6xt=!CmhGEAxxu?3EGyj@lth7NNi2l9V>2K zm-X?{(UoSxw?7DN)#tEbf(U*n>*;IjPk@lyxMBIoH3 zh8`==2BBp!_lVaYqhXxyTtwQ>7!AED&QJC8;qe(UsETS5TKcrpkg<$)6>)#7?!Lk< z+B3Yg;%?1st~ZdMA*jQ-si79^WNBi)DRm&`dE#Ejm+K#&Gkos&cIA|O2i02cw*wn9 z289&76qF?7fe`r@U^(cB)wmI@b9h5kR43SXT?+^LpHXT>E@o?)V(Z9ms-nXxOHD6a zFwKa*$~JSmui`RtA{2YgMK`JM@dJXs$Rd_<7YkM4_QDm+jT_Gz-&zdLwseRU~ zV2%-3M{r(U{#@I|34><6>3I{|hT3BCO19RLpoWcm^X4d%n6)DGk+;K=emZrGib{E5 zwCGgURZ}GDA6gERe;eNSY;pf z-^(`14}KC^c;mb=S4APpFD9mC)Rp;n^^+#Vg{`RBF8+PnKP)U|@&;O@o^!A~1+;6I ziRayWH`bE3I+~r@A8E0$NbvF}DE(#fn#q*{SY!@|DSev3XpGU_`6~kCE)+MbvNOh5 z_-B_HkBXWJYR#mG#W{IMA7g{whF5GY3OA^^@3!16+?^K0KG-D9Ams}YZV{GA+ylhavtzzvU!GjY5~w4y z)VMHFy^xmyKh_lExQ%^6^~^AI2Ciq<6~*O?0Rey)9hEZ&q0p3{Pu5D& zD(3oQNqvUXDZfQmj@M{9Y5ekaPb@UREiY_4o{}%fp$dRqV8XN?o1Feo<&4O5iQRa^ zk$R^`qh6lg-YH8%Z7=gYsF<2En!`DGS%X6iTA%M%tsGK_lRe!HQgx0wTP5C{*V&H#(xe*pNZjcTK~Zj_2aEuyw66U=B`u$P_^yECNG z!E-3*AbE(@7!>cnlndpUkRB|AV}(M0$p?B2pqp+Ef5VJUjbAckLf&6G(-SwtD@y_e zr(sJ!C`as?4{ic4?HltU*Vg@27CR}F#n~`X0UKvtZzCWIM9I?<1G+`w&p*|(W*V+TsRnC^x02VW z>CdP~k9=dmiAOM{F;<#3ZU}5U;?+h^iS-RyfHFg(y1pLQHFm14CtbU+DneqWSH-}k zV~awxeZX}5n9GXW#)j0}3#>~-0yWjPbA?SJLPRF%Sa96h)58@*zy+Qm2CrXOcu-tr z3L_tgQh43=aZOSych9i(UA@^=s!}n=>Prf`eVKPXwnDh93X5W*UuPy-UnobfcsC10N66d^e?PESl|69Ln3TdU{7$L1_^6Q`vyEM_0H?yDn(c^YP#h@|7DE zYPm1HP(6T>>~tHM7Pxyr`u{KxnH#nY+NfDBIkAH>ax)i>$%5_!)TFnB!U9=k98G-z ztsLA)XAaT}^@H8oQE=W~cr3)^^n`iO9$S0ggUW0qu@h1hek$fq!v^H<3%bqjH8v8d znecD(Xl;$bqI$6=2?sNK^6w7epFz`?T8=T6Xpyw8hW(c3Zd^nH`iSeQ^M-9)6@TyL zv?c@l8|>&j9)aGtDpNlw37M;)kaSs}Oh(OgEDmiJ48UaP94jI zQ=8rblGab~WOPyE&ddhmN-QU0IKyT>b;p@}p36-RdwH`$O>eeA-TPe{BrDRh4+L4p zxsH;DY;0WBEh9&`#JM)bjnMtS?B|^;10@ZvMA~M!=}YBT10WJ=nb>cU617U4W6R6L zB@Z0{Lztp+hKOSurzj7%Uy&U1)Ooec^rPM1XWr7l$38#52nfPmQOHot7Z~J)f%sh9Ni(;L&oz{du3K9w-;G@0EC*adWG)pU(@j9~z<6$fuwM^==9(uLH+Wi5TuM zZx-EcYzE9{@ZE-b?I>r~;Jl0+R?8bd3 z34Z=OrjztjNd&>wagVdBI+J9Dn-7a+yN_gUzPnix%AsDzRVPZ{9#^dPYURDrFY*<8 z?P!sIQOxm%qAw|b8qNvOp&lDf>$aK|f6sGeiLjWU@ztidhvV*)aWCX03aPD+g;fgI zO+a7FrTO^=7$`IYZ2eEPJMim4`>9G)IaONAb1*sh!GdyHJ5N}d)9%VJ@JM%_!&FWG z=zN;Nu|8Yhm2Q;5Isq6%;I(69QakHE_-jeHPpbEFb0XGSf{)bt_GY+SQJi5@GL3Gl zNK?#jDlG~X;_cH##iF7}YtZKHl*L;M{YVs2!p;@qETlEJ2rBjYto1lY0Vz7sR5=lB zZ<`aXc4YQXgQ}gIDGpg;AXP;yI4r{~t%nJOR}iFCX0r=_$9pnHr)|xLFMKuJk+(ue zLW1+=S}!K9=0S?`0djJOf}Uex4+h zpAG!$jB5`o@=jC%d6l6Y%9UP=!}<6KW?oLP($+GnGe>Q*0p}ybEpO6_wO(1ZUpFVC z{Rb1Zl|eDH{5x%H{Jpc#-x*0i*9%6=;u2SldmeD7r{^#CzJ{hB=^`u@bbgB8Ef$?* z7>a|GVQqFD>Al&_HsESQTj|x^yN(C~`P}CvKK9trI75Tt>?Z>Lx;EG`_<~MCD+#K{ z%N}Zk{tK&tj>P4jvl5T1@9x{$Ef%lo-zbBfzYkibu8O7?pAfosm2D(hwo{5(>=G%o zxZ_cKt|+OM=Zn)f{;M4-?N;-R$8OIDfQrR#X@SYDj?%H^Y<_r--0ArH|CB^PpQ!=0 zzC(c~1@0C_;D)!r>@a0!5(tgl>775m1&1U-x5BsjC-c?<;$bK8Ju$SmejN{SqH;8^|@^wYLK;8!~FM`C&<;o|Pim zaN&vFP=0JEZqn9);dhzD;ELcx< zMZ-URn>Mvwrsy?~D^fGX+fl&0N#6gfGg-)( z<=$iu+>`Mkorq_sc}ycGHLR68)orc==)WYFkfE!e;yq?c)GJ=~uuEcw&dWI8`q_(i ze!|gpIj^K}%B<+o&4UwL4pl-2eA-j{A_1H`UDoBm{Nc$IC;f3}`!z2Np?JQs`v`Xk zP%qs^54+&Cfrs%nP+1f#6kbq;4!5O^ENt#8YR`^qS@1m&0MgcK4x#SwM(YB-MCBlpA86gxv%K~#hvEog@I#pp?%E+zU<@F|_QcpRE7tY+ zt&DlX5%)LGKL%f2Y=u2+F!sC0sZ^mP1-^D}uM&~CxL32mH}GKN;(D9P5i!964BWM5O++_ZXh|ll<-L1xGhoLUIsH_w9 ze#|XOBKgw3gF}y7pz91Z`9aHIJN6_?V;oBf4jLYzYU;RlYp#^=7V8jtBg@98q~hAN zQH4ocw?$toTu9W#gnl^Uh?d|N!#3z1YX#Bc+2aNoVT}wXb-Zm{T^zR1Fh%N*RzPq2 z9=Fo;tgli={f+5tvme{(-@CV}U&GK??@x6fblw;-G4o0kKsqAu&KnLj=X~jL;5dZsxV_I9-$LqQV z#;M)hptv2I4~?is+fjB*XZH-lXoIZl(HqXA+vnLxW)8`yi;5etIg7mTB8R0}TV@!z z3;d-yL$b9^7=_S&9DLFuh{&+*w^UrJ@vcKdk26iD4N-61UTY=9-V6I4`K$ZADz;w{ zbO8sN-Aev^>~sbJqHH0SzMmJ;89H+d3#+~_uL1Ld_#RfQ36jrJb=Zwp4ZnHQ3+uVg z1t@#bZq8hqJ9HTSY3CtNC|sO0#5)sAVHaRi-x)vZuv+}4Uehx?g;XXBM}%5{cx9iP z45x>ArrlII;loI7B|DeSTt$7*_eus;E`Eli$I#|lrs3+CT7k2hLeB7Kz2^LY?lCXT zXlN3IW)w&>=<3mzv>)-#k=9r*EsLm8yxI8b(g@u4Sa+{&rRjx(J^wjdvrMipKWss8 z(j8w$Dvzgi_KkT7&IT^&DIZ57EaT{NCf`^ zW-?a?SP0(+O!azu@_b{~XHLr@^Y9j1IpKL%*pyKa26S|EIR5yfyc=BJFMvOI{XZTP zEkrL|vx3TRt-c0_8kVE4n8KHaH^4!(%Y7-q zd6Eb7go^6^_VAyi>CT%uJ))Wdr@T<7weQ*zCdv z)iap|ku%UhD;26mCxtQLGALHnN`>A(>s#9!<9%vt)nl;VSp!6R3bfmqoeHO}q)16x z<%bVdzPW$bqqe3MW_ro2;rnHYx8jYTyrvg*=h34~waPspURgW7dnoDx8Dy|dxLz;(R>!Um!Hq&Pz zMod&2eZ4?HNrWUpMwnPtcr>LeWo|iXfGOG5{ z-G0aYoX2azG>ElYKZ~7{(+g9c`_Ou(HyWY4L)(^tAEUcb8om37SAc+Z6_kERKwHC$ z1Cj&|b-06)XyJn}>mGa`tBSg_H1Nh>k{l)>d}rzqsCIO>1yy~IlC@!vzzPMuA^fJd z8F1n3kXan-bICzDHL@fKCwGQsB+AY$yCye)%IuqPGp*<_I49|pdOBt}MzqrG=J%&a zWr5o=dJFM*@=}JTLOW*8e!TdwCw!B9E25z#FD5xI?R}1>7|YxDF%1n*E9amk-2g6- zEH*G2f^Mur;r+H`-qc!kT)p{ZBHZ2iBfQ}RNJ-b&*fhTd9wv)CimsH9_0&7oEN?yH zPVrsyz5WT0;GrivI{#i1GShtlY!B6&df2u%{(41u`pK`l6gSgh^P>u}?TxNRey6CI z(dWjjoXZ#0!C!%a1SSn{s^IqBI>QeMYI$CUm6Ff&NMbu9V~pyRaPDG}bUi2Bk6;W` zwZA>a9K}R{s|7UNBY~%xvpb2EjnoLA`hG$Dogq4zkgL{$qmcRUyy^m+KFD0dd#ISS zb`jRVwYr4yock)Hx`)Z)uL1f0uGN0)Dc?2rcwbH+bQN)YDe89Ms=xP;cUuj*i^R~r ztRciV&UfqO-J~)1LinJ47zg5!wZ)-jk<`pJasnO>*h$>Vf3ihd^V^hpC3t zosh_k_vZD&(VptkfP3q2?<`S~D^WmC{%h_lON#o8aXac`;M%T#2_n%^$HxHGw(zB> zWzPPbVxsi4v=sCcFic}-X=09m6#)io##zwo{zJ`Qx30X@!g`4tB5Z|@<|FZIa2Sm~ ztd|)|925p0yPtOXJKsh>61q3q!8!+A*3660IMS?OUVfEj;meY~s0*P8D%fbkEEp6_ zKw2*4djdBN0Lwly&42tqH7%fFX>+}gCCM&hG~aGq-Xw5;WH1=PkdgUbCBz2qzNi|& z4Cok!16&2d`lLnO83+Y`w+1I=a&@zY%`gi?NoE4WMQ4A}GTHUe5tlIsFYP#&{qbsS zvY`!QoZvh;jfa>d&O2DNGpnG~eGY_gNVoyn**O#>ykRyvcl2VH^p!WM5&B?`n*$FBgOR+U!GC-_ciw=E4D$=9nk27-OB{ux1jJyiQdPK}{M7 zHY67yGanK}EnJEIibG~bN2*Vy6zTfBJaXWr0pb66Jy*eay|k>W5LmBT&QijTRn4-A z7X{Eo$cCs?l}m+$^!UKT@c}>4J@h%mDHp5RBJTM5`uVj+Uf&P50(z9Z9zZC8f+#FJ zL(Kx6uE2EeU=Nci?YeoEos<+tsjyTNlZ*EZNm3vfg4MVGe}-&974Up?Ym-S5K`xKA zwRa9qGR$Mt9M{!S)iW&(TdMr)HcvifJ!C|`9UG4%iyn7YVg&>Qpe1HrG3Z&!Sjsy$2 z3_O+z-gxsH+4~qcR^UrEjnAEgYT+DVJO2obPWLb113UucrTLWP(kM5dwtMIc@TSJ3 z8q&?9Pfbnr{)Y{BF;vH0=>~Fn`a-Z8ZyM)8wTrO<%L`36$ero=%#(}pX$d`Jl>RUG zcHhpYLH{X@M!8L?A$&iO1KL?FDMUfk@@jI8KD;Tz!`KTR7kfm2?G9 z<{WjNGPT51!*BqEo)OlU{a6pc0K1|0T-!Y##0) zhyOHpK}Y9AvN|&O0KAotIY+}PwO@A<#)M$03Jn1BRjec^^Qxe4GQjLPyuIMSN34gD-d3mH%) zb6ym43;xyo$cez)t|cYNa%hN6HnHqnU@mcr0g${7&5}pakPX;t%s{V1fs=DXse!j~ z3CpGR@5DUnW#p7Y4Ad1RWM{*Y-*Zbq*k`!y@N;vlcInCg3_3vZ3%C&y;zvMpvFm@S zxBeEt#w`MkM$z9blJoK^mjF<)Wo(aYSE5NxN@Fks5PYY-?yyPg=eq__9orrq%YJrp zYu!%OLK<5Or83?Pl9?WG&O%3Gho~cKL3#Tx?Em9}$nXDJS!rMJ@$A0mdWW6FWJ3;yX?25C zN8So@r!0U^0ojy>^h$I^FNnF=PrZ#QeokOf$PCn;iCa%#!s>wX#=wunM- z+d*Mr?;#ixwB_;-AC?{vleOEQix?BwN8HTdu+WHr;Aei7zA>(@WMFsFa^=~>1OwQ| z0l6g~_YK4H0Mulb$D2sk9>o)Q?D9X%ZGZp6T~6}n=5wXjg|Yxai_*Sr0LlXtdG76{ zDRGTph%cNP&0qsa*PZcHI6BGR5;waBecn9Y6x<=TO4{9j@x^L#_TO0e=S2GXnvtVI zi_HV(-X+7n;@S?&y)43Sg`5YUxEn~RX0&z4j`wEWC+I$9n$4|;1=S7bUSK|OCks3& z6pA$eG?f`c(>E#viRYRLhR4$41brl zmfO(Y@r98G)QpT%2sI}&{jD2R;Njt0AG@7^p$h#a+jntH@)#sdk#`M_(c7jp+Svl7 z1mj+F4rY-JmdcFF?BDR6kW0eQif^WAqsL@3;X*-{VoO?4JU@H{m=&6?IeO6s~O0vvrh<;kgx$#)UXqh|Kf%G9bu{{ zDQ{Y(DwYsBXalY@(YL)XmlqhF?YdUCd?g4vimT#rrW)Wa1_cfKrwSd!NNK0D!>(+8 z`rSMjostOSf-+Ql8gQYj0MDezsSaFSf=k0trelNkh9HSj=Nc=cv3ff!%1F@3uC5i8 zqZy$z+RrWj(?phiFxp4732vg^B9r!2NUb5!)K~-oBxzVd&aU7nS?inKB&(H!136Yu z(p`Ip97|mTp(^7rbAeIdz|MR^neoeM?1(N;9uqALSxRo@&UUNa0{+5jY36HJ!A!}T zbMq-k3apIGwU7}O1Pv&tIMUyuB2uu@y5R@j!F%k8&yX(9M6*wy3S=$!+nthtUE66T zfHF1Ta#GLoFalptiUCCt38f7aLA zxXH-_^4|+wyQ4xHPlZ@y)2Ulm$D0kK3r#y`ba|dZsOAMEnYrBuh*My1QRkuAT{zDG z-&LAst9!bSn5}f}2&AX|G}2yMGHi013hSaM`-LRO0zhb6NB>^oTzJ>?C0~ zle!1YGbD+L27$KF+%C0;m{xYE=*lcx=aovh?L1lEAH1VU-Krd56X<+C;(}N>{$3;) zK7*w;X(?e3Dwoy#u=Uu-%g}BvL56J_VNW=Aw+o}y^;$eFsKndzU2#Z9N*;kRFx8_<|ZFMhGgE*lifr_m7lzJ~loLBr8Ze zTQj3T<1wo=^h&9Dgg-l05-ivCMq~^AZKyx8-%4q>10DmZglz$Jq-;l#Lr%Eu2DjB@ zi)wR(OE1(S@#j(H+2YRwbl~YDw7vZhB~t~A{U3KLOkP=VUIaS` zer6UiY~+yGwc(jDdC<<4zi=AX%{SfGd!Euo3-sKMo2INHwL=)jiGL6#wa<-(&>B1vB@4PBRjhQhzs9~y^arD`q5n^UZbB2x`u zbwoRC>wKO?f^U7Z?OHtsSM3u5$ma4Td{`1L+Szy|VMg)Ln_5XV^jER_EPU`5{yv^A*(q2Ct z{q1RN-yi7C1qczP#3tpP_g#O~|2V@KFDSA+l>u_eFsME!U;ZD??662c=z0el&w;Kd z?S4wr#)OB%0Md}f+Z%h}%IdpP-k8yNoH88L+krsrM;I!pH|(?5IT!MIzMZN7E4 zBW63DtmQ77TzE^RcOPf50vjaOErw56#6LSwa0p$qLJJzgc=spCuf~3G z_xZ)HWpY$t3fzo$L7m;FmJ01ey2x8z#+q~LZ4u@d+Un4SSN@%5Msq_!_BmRQ4U+l) z5OnY~*U7OlB6@dM-wzCm!to(ie`wvSlc5P_=xk9D8@YDDacZ|W_W8+Ga0d2uDe{a! zOc^{TKgo%y59|~OfR$127*oqfgKzG==u*!*FyjPqlA@k5v|7xUkg>!GIp>QV?VUN~jVn0|dx46BBwwMQ{3FXe_~ z){y+y{gq##>i}FDS|QkCC>Fp0RxUteZ#I1-U#`ud6Cy<5fl=F;$13hkYbp8FEi=yW z{z1VXeupqY(4+VLGrIHBuUCH0ETBgQ_ktALrAWJnK@DD1?hJKay!4hwThF#Ql?z zMbD$&opYCUqv0IQu|s2>u|0hk+G5+=t8N={HWuY3lbmw;qO8O^0&L#?TwT{pCjB{( zk2@8U;P9TPENgcTUCR*&jJ!q<=HAT?NLh}18>)%q)K$sRjxP|cnFJ;IAC-nI&1~O` zaGZY)e>d*Q3np`k-8s+Jmy86hCU-F>2%9{+%iiGS0Ttmhp&+Y+WkUEt(88Gy49}x@ z454(d;~8r8bY%W-=STfi1pP4n-@g4*`&X^WsgcH;oMB`!J0%deq`&`oxM;%OS^mYpzBY|wNoR- zD|PWDt#BDurf(dv8A3wub^Yl-G}!++UZ$vC{C9IlKH0rINmmUX-wMro0`PHBcHX=JuC6l*62eog@t~3+4;0@i2gvZe6e zG~G=(nHcQn%iES6|O{r+v z)k(?|UV5xd#)W!%SjvTBIy{!(7|LSMP67upP{Lcgg59`d7KX;1EQ?b2VF9iSI= z%!RUh@{N|urc#n}%6+f2%T{uCW+6f+9^Mj~^$@H1F z?M3ye*~3078E(M~fv3>0ZsLQ6T+qA{g*4{gnJIPt_sEF~LC-=@%$}$ho7s(*{U4Pw z)FRO4*P(8VG0*wTCaY4|Y`lMaVc$sbxW9~4r*`|2o8`%W#y3DEow~P@6C^8Y@HNw7 zYI#(pE{AyS)=&7egYh3q6tc5Mtpxw`95N-Z-HaeMQrhtX?>6XhN5Yb9J@9G$HB2Rf z!=14+OG5w0Nzy{Iw-+V&l6r_!aso6nsRZ5mgvGg}TIk(D*KQ|=@OKA<8HNvlH_`6= z0Z_CSKtn5<`_Kg$-wvFNwszLpM)!JmY5)(FPwOup>g<61pD45@J zUSq#=sc<8I4wah#;2=)gHwFnM0hIK&FM5A*QxR7<2O?8s1>L{DAO;bFA+*>E?n~IN~81F0_>rv>+ z2Eb}p8L?lScL;G+tP}VH6qd97*Rj1sb4t3)4Mfl1+F)ku(=8`gucb&J{-aT%bKp zMX}WQT4A-f_$ZC}pufM_V)A3PwZt(U&_sbE9X%#LAZJhLuxpKZcJgN;!GyxQ#;?G| z$TC^66mJ_N=4|d7VBX+CfxUj6(%aN1sy{N4hb68M$(qxd9`hAg9x6_}ZG-JAi9nqH zZ*7}MH39^@kHHV%kN{l`kKj{aG=^_B&K=ful?r!5$ z)SVjpZ0SfnOH+z@&X;wnF@hTrqj4Etm_RYhEZpE~V5Mk`ID{_t&feAAN%Kq;dP;9I zsVQ$+Ta5vX#F@<3k?^${ETfU(LMfuM^Uw;1l8T9Ax zwj(dangA=XB8_K$4OQk0j#jgQ41*BKYJdGCJzL@z7M}MQ;q+bRgn3CQ$44|9w zrhwldSx*CPgm!@(E=h{9d!gp#iT&jO5n}GhwDt5AEH6#{EkC&eUz)xo!ew{UJg+76Z!3nB8|V6>5fK4uNgr80a-{0%t!^F@ZGx#f(u`VHKNLDw+#-Qi zR0#7?hH#?&HK;jYgM)gca%N(XrC${-J?oip5zM#Ma-lY5o-;j!&oy|eOKL7Zx&tG> zpFcLQQj+VnwZxKp1||g4APknrRdA?X%cV6@DXHa;i+AxSrM4fQcJu9kbfQlqrHaymY5k<2h zl|XP{7U3xc2SmU{=)afi{HGZc4o z&0E+ZzFo>6|D_zbTg}aG#<+(_p~m~Tu?Gu1F7TF!YIQ&9s^a z)bO7Gz)}nFO@yt56gS{OX16{M!!c-f0<(bgG3z%(`Gg_L(^ngEJHtSDcFTwUQO|zI zBKs6>W1!HAoSw-a7GpG1)ZdCs4Je?|%+7mibBvFFwlU^8$x32T!lrgd0|E5@mk&ZS zEp<0N5_i`tdn3%O&$Mm)L&nbj)%z_1HXn#_S9&A3a69cFSD$>V0}e^kvN7fWz(%5SHVpD&4n*9k=nIb0q@+fh_(cEV5{YnNmV3cm9kDIp}VX zhNx%ul=H{FX;VFv1N?9~E8h5{Jl^TOVX}sJPP-0G2@P@+o)ih7U{{I#9lv2IXYLf_ zI6=RM#02lm4?0JtV_wU3c5&6S9w2KpD`*m~=EC;oJ~DA{#PK68nsuU*ft=%vtBF!- z?S-wd8s0kq_rB^e_xOzp^$Z|p@|gllQnsn78Peu~{0gbOP%w~06f5uWdm&7RN-}_8 z@p=uUoIb;IZ$*IM32LwVE;w!$fgdoP?9kExg_LH|`Ju{Vcu<8jWmtAz8|-s z?-b7nb`aT5MpSH_pIkx9Z6#$kXFDZmp#K|K@UO3kCxj#dWcqk_xN+p!8n-huMYb&IbRWxiJ< zEsCV-wjWlgnrQt(i^6+O)j{P8D?e}72r^N&UyNt`^|yWhkF7U>hqC?tfVD{R;Auf* zsZb%5EZMcuQ1&frCF|Jtofau;$x`;R6O$PG7Fi;@G4>@Ph8a6q-*Z#H-|zpv?|eQz z@x;u1U-xyL^F8N$&-Y*gU%Y7h&R~iPLo(!`bJo_Vfj&$Lt^fn9#9#f?F@i8{1tA?k zRl{j|;9h5{B+v0-SLI2KO^=B3?=ThWb$xQH)Ehwg;8G*KGst4TmLN9rt44&gw-r(BvwDNZL< z$yCTD3HOI-6j$2L_P_s8_yrtbgx$xg{-~;dL0`O;no8O1(wF-q*MCFNEq`v1oXM;H zCl+a+JyOB}ck-%qvCS*EF#W@SYQd%Tizea#H0 z>)nl6f9t+-yDgtF4{6sx_M7CD<=kTj)WDdTj5y(cMTiVVasYME94dcI;_k_vZ>wqX z;i+3}eT@CUVCDi}j36E(got_2M)C--f)HF_`}^G#zx!sK2oOk&^bi!ij&b2{aAE}Z zy2>PcT+pOHO~vSwkF=zF6}1MV#jl$^PYhJ0RWC{pJGVfC^jnDr!>u0{S&#TZs24{b)ax5St!i$Kr_pDu*lu@-u#mCdZF~ zXVr${rrJrD|EVNizY&8(581y>%AqD0TUY$1tvqYbn}#JLM-jYt;jnwf(;vigX~f3u zHF}kP4f(p;RiimdDuYh5_tN=+sz3v52cM1w+SAw#X}Bo*mKd)uC4?BLbt8QoO5dUR z4JwoKeXW!cQC7`S7J4&to!Xjj4D!P#-w;ddDpjwU0%Qt+-rOm_ zcl?4k{_{wy4@NrB2O?+xmeEKI&+$8=>#AW*0&fD5r(cqqgu3WVnXn>u8$R+(FQdolm zDEmJ#uyRdWC`TwXDsBIM4VVk6)Wk~DGPU|a04!2sw47aZ2o;V)H6;eKs7abSp%@nS`9Hs^RTI`}&-V;JoiL zX&Y+Y#Ao)qCU%R0fH@>fra}0Zun6$E2lV#xVTcSW?40ygSz*(cTYJrF$#+kQOwYY$ zpYBKrghcx6ti3O#`A}JLw21U@*Tw&+XB`4Up|a+!nlG@s8E(Dp2L32)2Ob!^bNC3* zZ-)4&$dnZjMki2bm_$s%p~e@xf)G`9>9+AOeWGM>MM<#>DImV+omujPLLDnaI56j{ z@_VI!4G?CxWAVV(x&Xh_JNp(oAd9jr)JQhd5@%Tcr-l`K^J(9d{E=*AP4qA>m8r=U(B0O&b9Em zTJ#EPkeG78n#0o4SGH;e{0o8I(bD>8w-*iqF6cJ}IjZZR?v>F-ia_eG>+>6RVn9BC zxgNGmNJ|osJeu^h&OmYiD2#Tcsp3d`1LAMngnFUTA{QE!CIcr}KjeG{cOUt26GE{_ zWlXgLm;Xgn5^S4ie1d^fdx-T4aSG>@wyG+yv znIV2~lIo-)Vd{}xlDhes(UjIH3_`$)}&M)qn99Rel znwJI)4RUM`Q9KB$iONPNOciEH=e~GG`pvz0v!tNKC48{zB=mZ$TrYAG$fu7ZibOu%V;fvx^<2< zCb`(}kmLhlLjovy zYY<-CCMS<+Vh(cZne8iei30WL6xnOn7xB75m zuBR11uQ+{id=ST777Us}){__iNdCcSHa@Kp5s-KXtrrDEkoSMYvF~_ee_$RV=|1Fz z)eGL~{}llP9d|hyalQP8j-w;jfvQ%1Wy2Vm0|{(2T%O$ZcVYtj$&?L%>Irl#i#MO$ zvk`H?J|KZ>LMKO={xNQIHQCw&)i9=xPP_8qHC;euL;4;$eJxi$zYC2&MGs?OOAZ~t z@$&FA^;i#A09|RH@{RwQYt^(<62yQL?r=Em1YT(UoN+|G%zR3efrVuy2(K7zygZ+V z2Dxy>R=S*Akaz%I@H--lPpMjuD+tVA_%rZ^Z(nyfrovVys|}2C_+LGh>tW$&Y!Qp@ z+5il-bD<=Ge&ffqolPI>r>**YayVNUJ>g4+x_M1qsV(RpChBAW2qHyan$S!`m%4YF65rXoq_R%gJar$c$r-<(pe^&XJ{w30|IkXAQ5WEQfNJtt|YbHc38gNY3Acj zvSr)E-BW@#EyJ-ntMK-SFihybwmddgk#?;Me$yqMkZl_+IA4x5+We$=m%-& z3)AJR!+2Ae({b)@G|&K$6_?>vGdfTWW;s;JYj@shT8m@nEZ`K7r9|eMBKfb0R=2oI zVAqZFA5EpXuWr!>=X~)qX?8{UMninG5#e?hD^(ZAbhS3)Er`*18s6S;0))m>$;d@c zpZv#F7wCbw5+}4x)V?mp|DzH$K5Nh6Mik`vcG$^asN7#+1P7Mq-M7$iapf7s2Mtc^ z-?f%u@P!%7@d|on-flZh#oyZPBc(gh^?t3XHA1e0r1yKq@SXtQsk8Sq7{1-AtV z-R%nFo|L$Q2%0)BL7UR8_)GI=#&pth48}osu*6Zd9WL3oN(&(DuAGzDk$-%9P&3$ZLGFPF$9Uw9nG@Kj{m4LC;nK<}6kyjVV zrYk_7SNA(xK_EfszGcfB6YmFHh-AQ7>0l^<_mcZ4G9#09?Q zSRWXpcaD!qN7!FbxMX2yVSi=YU579yt6c#f^ijz5nY2@lLf(!25D+h z)`~*U5DE&Bilzn)N>-eb?C;b)=MkDGn8{Y;Hi$Nf$ z>}!4f=e(4VoW+lk%!pQhy%Q~K!@AE3QSxG=*v`%x)uD)Q;fxaRQ_uEXt1_C7_E+aX z3(+P=W5MF$>>COvVBj2LSQ{pQcS?BRE=2(_jOU&~kRdL)CUhiX|Ind>)>V^ZB_$=36{OZl@8N`ZsA_y!eGjXI<+Jv8J+HlG-)sxY8`UeR))Pvn zGwHK~LcUa7IJqVn=BXVjakX|5o7X$TB(_OI&EoBE@NTtCcM>8B-zJsHUuKBJnpI!; z)}+bqBQezwIK7FX-R4D6(o&UMmQK>Vn96b)+LHOkm9dVe*y&!&O={*l0@9W}R@b5> z9ZGtQ)n6RVeJXU;`BSfP;{HdQ(H=AC3O`7kmLvKV2t-2cYy&SU+4NYd|>!>YC zq&D^UdzI zRF8$~hqG8-zS+D0sw2;VRiu zgg^Ir;;|MkzJK*zbxw6{z1jJBIjB$UW3Ye#bILUH3^v|w)QPF3SWL1^ZW_&Z zNa>nvEaMHa6E(Q?sWrPTdtxl25#o_LfOfvOyHS{u8-ZixOj|Kfd92L*_HjRdzm8i_ z@7G(=wB+?*7_P8t+GZz(1v50xvQ{%~Ol2H>Oqm;z9dkKgwR&;hee}0$p_in`N*B9N z-01vfyU)oKhshSVIE>mPVHvtk-6e$Ax_=xi5 zN5kLc*pS+*JhS)|vDAr>O^oYO$P_Vb(#+1k|AnV+j8=M6nQGXxCN0+w--LJPoNDC3 zbX5*{DTO(LM$p~ZOG9aPet7iG_Llq4(GI5uY1{ji))l|$o833<5~f)^@K=MGML*vm zY<$ma)zIi%Fl!qAms-Px4eQk!wQTPq`n zbT%DJJCH(|e0Q9@CryrvPMvTe`nI-N+~O zTlE!<2A*vtNv=vhl1R&$v1E}bQrht(P|t2UXVts;Q?sQE4~yN&t{JTPQgJXqZpW^y z{as1&s^P72TKRR7MzI!8kZ1Lf`rCepN7>%FHc@G z-gfE5=kjlFk>u}eerOENPG44quyUJQ-YR2NSbrxiQ5I|vEkgRt z=rQPoKeD^Q7SvPrSCf}gNIUMF$ZN_t##-VNa84;ODG>6iMV7_(_E>hTWt``%(^gl3 zTkrm2#>&X862C$5#DeFJd>-eG$mqU?u5yr(K5AxtuqnCKE;9Hb|9E}GT(6E-zPsPi zrHjmKUMsc3Qjglj%7S(L3)|U9e?!W)g=k)Q@M$huBaA7^I)aKix zG|pmitm0G~B~q~Tb?!4m#pMm6Z|Z`>nprImdU#PUnwVTIGsruwgkSRGJT0R#E5@Cv zFE=PkY+bh<_t3dSDx_udDs%TEGOwD>f)P@ZT=`UQYeV=X9DQlZ084N!uMepwUG(nc zJ#Fx+PzK1AMH2dUxAgVBmT}YVdH%gAyY|WAWaYrXfL>l;L!n!faBZ{e_0b3KbxLd> z_l~PG8l6{Ev6s*MF)8()b+9yAU-2m6n6TN(fUD%1a_subEZSN0!qd$b>X>7AgCBRm z5?6Xj7dZ|~=QRqhCUuO~d@HGFT4uW1wi;4$DXM9_<2Ab+GP=&o9=(T4{2mN3)qYLm)dwt5rg7#5pDXQa324&cReV z6Rf@oGY_)~pTuhT#*ZMr>Wvj*$4-nu`&e%EBE3QB4 z#L9udyZh?0JSA+|M%7vmS^2BfFQ#CX$5Z9Egx9ftft)Cp5GFVC02(+m*_W)mY5kvbFsL)wy!- z6uwO1z}gSOR(n%jnJ+Al|wz2&WOK% zPE&aUj0r)S4bwdxLDF<>zAQp(Pc=`3(>~GS-F3K$erEqg6o%gwRX8Z6bLMUgmq`_A z@H)uKt$Y%5oc=(Rbmv)5)fLMX5_ zte}7VVfcj3te62wIEsBM=&~>@3{0U}UL~g*6eIVblvIjdkSTU*#`excFU}}L|DM~b zs{sE#NK^`A^_ad_s7xGKi$Z4Wh<3C#y!KALJ^9m9jm`6Gy3pCoa;Z(rq>q~D3lqSs2T}33bmYn2@uCU5SFF;p)7ah0{r$GdGvq=*W^^e;6nA+bkjkfmXdM-S2CHTPnj& zFye;NZKgcXf&|QtlCa08C)@nCF+O@>U^%-s&{BAw#dsIO49~dJSls#udw>N#SC5p| zPhB2_o^ZJynXHI+Q42Nj>Z%x9z7>C5LB`j>a5Kl*azj>E*v2sn3n*xr*l);!Ax&!% zQVx0^p@4$Yn#p{>4VutxBlyRs_U=|%WP(+VJE&p`EF(CmOUBRTDP)8l8nk~?K8j{A2e2J(r;WnnSL#VMLbnW>K@ve@dA5PQwn^U1xnb_Kr!Z|V;JD2)y%?H zV@9W&BPQ1zcr2Kv(pjSo-fUUz}w=i6UWCQ+&>;~1rF_kCm* zhtjSn#1vZcboLxKKR+mCrWV(0bme z*6SIw_G7Gir6>wX6j98T6Z9U6uwuNW^_e|<*uxYDgp|$9(sRAwTJoB%G3Hwj>V$=d z#aix>TX!$Y*F5v4%-Y&ZL6mA~L&5)CP-Y=q$8vYacmSiIp52xBIXAt6rWDBy8nc|E@4b~ahqrZ=@&w}9@HL`y>DNE!Jmozw(sbkUP# za;mR3E+#fC)9J{m0Db?U?|1IT)2Dr@A$0|4uzj2QzVQ3EcJaE=pnJdB;A&fvF%4r) zo)uRL?O}hRmCm3uPLHy53r2|3Hi4j?d@me8Dr$eCE%tt|{e5jW%iiUYpv9&TE2@LDJ4NHEB$KBfv5Ho3|RZ42kpePIa z=~(Hfg?FZf5`NYG#?UdcSr($Ff4*e;x3jPz7nZ|~nyWSXsMe$JASNg1VO{oQ)|pp7 zU&iSCoy!){m#l;6c@55?&Y^sMn(>W(eM;qyOE~=8_EJFAGTDvvt{ATHP;S0=>I_?le2QGBf;E_xtQFzR%ezgqh;`_4zfsU^elR+g4LF{s!yyggQx- zYJ!1e5EJrTXOqqb1c}?ui5qRK9$056bp{Ut`|lriWtQWEas`Q}v$e6Z{4G9Ece83g zekJYH<6+Avh1|kn+67sFg?~G$8|0gUbR0O)I$TCBJhO1}a+jyt>QO9V8hs&0G+)t0{km4d89w*#C zF7Tw-Pn;0KCa#k=A(8g%D~F`SA`0$BM%BPpRGN7J?zMbw=8f!Kpdoc_YSCNNW(py9 z{m}m(dqPPJ|C!6Ed4~jLLR+M){_jE8O6|&aBbl|p9FJZv}A7Uy5qHnN)BA4U;Z@&(lRbiRF3>QWp^NBvB>o6xr?FKWVwee*umgw z16?oni=Wq`4CRK6WB5uWLm*>u#0%Y|6O;3$YCIJR%_dR43in4_b&wTH7bY@^w9Fhs zzD}&m+--HVh(^rd-1qZ?)#aDly#$??W@W2zF<`}lX!2DJ(nd>(WjCH$7iaV+)=ai+ zJ+u@-GpYVuGxG6LMx016)7Fa9)Xw_IaGmoP+fU&P?q2t&IXJBws0MO$^16)}M)~5z zGv!uutT&$jQ*sO0X*`iLj`J>Q||{?7&2g-l1qwc(LDQdvw=cpH7# z`1P71&COE^_M(u-gd>c}-S?@Rfd2KQ;bOP`*7pv-CgDB2p)6q{$N zD6&xi5Dr>f6T{S0182nu&8fV)oEcsErn3d7YvDX!e;D?|MB@Egu~HGa%7KfKXCL$= z)q97BGb}7S2mh)l$36sQ)Pd@AL8sIn8SZ$CFvY#3W!`Mb4xD@ZbeOTf+{2k47RZfq zfu4p8Y_n|Q+QeIL*2y3=Y41ars{yqo{;l*7J0o&6;j#8 zW4Ns4d)4myt2rCigtWNQyr~E@Q_P94E3-7-$9ak{f2ZOY-VL^CF5p|fl4T}PG4K^_ zL2?gyT~Gvmx~iI*(~qxqTWd9;QBr5%=p89>CZzHZx-|kLdFf7oV6e=v09R3&dWP>M z!(#oiCxkbz-m3FCMB*jFmiCfFYYGX6$t+$u`rjyBozaSFVHv#sxx)6U02ZDowkWfB zhFt6?{Un?n#vQBBnE7YPWr`&t^-F5N%?sST=fSa@&cQX_pn_qO`X+5!-y+Ex%J9#? zL|*?B#7rSu!tmqpFJ}@w=_}LRT!Q60JJnw~o%CCxUcWQ9tSo$pl~T|)%oJKZ*FUZb zTOpS|#cqA66)~QL)*TmyT+ZuJqG!@xdZDuv-lkO7l(bM4_j!F!F<#ZCdzgg2L}vc8 z;Wd+}{m&GogPjtKmcKpI_nJ|wV!FE2y|$K3ygsiS2MqWfyegTVNyTP|z1y%?*YJxNn$`0JN(@fFT|dsH*s-|ZiUQup%bfc{_aT;StKoqV18H@ z^8VRqF}Dxyt1|{O2`)AMgPH6Za)|f46&PjS{=+!nB{6BV#+vu?qX94>k$ifILTKqu zu*DqTez^ix-(V4+pkcod%cgVJ<*NK<+ih>tq3SZPs>zU}HOTuTrl0ms90wPv^)pPW zWL&)$wpnFEXwyvbrdhzK#pX@6(0Od$>;qo?AcJiBy)ba=gHDqywcd-@z=t28sJs%t zjr!xv${G<6JL3_XWg*}lJRxA2<{fT{peA0FK6OEEeX;#(1CKNYPTB> zch|%u^~YF3CsDt2U?ubg%ZJ;vcY*Ir#$6p+A4>R@%NFeUZNa!H-?LXBnDvS*lsf${ z3ZnfowPmidUwPwedzsSt7Bc`4Siw-YZcfWK@P+Pk@2@V;>Pk7ix2LXfdezHGCKNJA zpFD3QN1$c5=>&AL)KbcJL6j4>cE1kH$UA#o$$ZDr{c*R&GhC##v+1(B zPo5c``=mT5YLWb5V|j(?J2e>i18V2ZrzMX&jf}W}Y(bQ;XYseg%!as+@8*Yh;D^Cm zbMM6@207@qj2q%c;-Z`ykkjca_P`{Uy|TK=<&B!i_OP@2lhujLSOLK4`p>H3Ns-i^ zR|TE#Qm_f$Ih>@jN&57708Ze37K)EAjkoBO+CMEh2vP4PEm%&eb=T}(W3u!|NaHb? zG7HO9TNzp<;4R|ern=I!Xt5T!CpLGWa+!)!MBzHdNjzX6nxc3+fxIB(vGV5GvojSp zfI&<72+Xa_9Ye$uVk($10K>wPk?r;M*L(@-y&Tqbc%N~p-mN-eYV!{isfw!AeG$<@ z?22EUOOFZ=9XkIl$2t}R3RrTVB@4U-izo<~uGzyo2et~TV^fQP3ExEeuB*B_@_A_E zMbDYAJc=hIck=PJb1(DXxZ7SHK@OVw!2|t8j(_Nw0NNlxkL_7UQHV)hY1!jjvy4 zeEv0zDdaH6`6o0?eFuVCC0pBY+B|HYI%n@@@@$2k#Q1j5S8;z8+j8+<_G8(9-D{#^ zBwF6h+M6RGlUQDm7ht>Ef0%M0Go`>n^n&F|7+CuZVp}p<+k}U%aXM@1Mmf z)Bfz58}Gh=@>)1De9<=(2o?&V&9PiuBoe{Ubp^%r6c8?VnwR=B-P^7k1y&|e2j6JC z^!+ru<88Zg!mQe60bgIYV@4)}HGz$^+0(AGynU>;)eA{v%DY_kWQt)E*j^iM3Vt=u zU%APRT7S3|1;q((g>Xgy&Ov!uUt#b){=c5O#C7xo zhxyH!`-r>{f()|@gEj9svMtXH-8F5E>m1smfFOwj<^F4%t4=}_oyhvuz0e?bCmKnM z98Ib}H70D;YZ1hFRQx9ULyD*=RwI9!t`D_MLB&L7q0j0L-=mMFamIm{jq=|QL{VFi53*pWH>ngFgTz#yj=6YaP6dMgItOb8< ztDWvsSah56?a#7bX)dqu?!jmT4F0hfg-#dBLQ<}QSgu7pt<;*#QyZ3K_`7OGRay25 zo0mHL>6#JPJuWiT2%`_>=3}U`M$(4=mH7eQ`^KTcv#528G1e$*eWo9o@L~i3@s`tgdw5RkQas1c!U#fr34*7{kb4CpiPcvxE)NJ zSeW~rumG0MQd5^~?d?b%WBA#xsClfisNcSAq|d^tIdS?D*~o%Qsk2p~PttmCxknWEMC9em^l)o^IZS?s z`Xzj4GTrq)j$f2gha$Pn2u-nWJ{9ByIYwpn{o%FYT(js!k1LlygjVWvH1%?SG^G&UcPf z;vC5V!C^*kiaP2yA*@mnS%bZoa$KCW&kzJf3DD!zwi_JDr|3V@%~SCGw*NcdX~Q&{ zSq{`*f_i%xA9N#(neiR$?TDp*&w&c^!MM1hbj9;%f)v zlOYd~CAE1|b!Z)ap{f+Z>+;RU7zFp=9l*WULPn$K%+s}QN-|ccT$uex*!j@V07rfA zAt8(FE0CeOZ}h*W`|I+lEaUoug-D2oA)NsDHUMIn-Mp{*J&zR@#5mnrncVSw#}AeY z>{tGDjnb0pX6!L>$^%84`Q{GTK2Y475wJb7i-7jFGV)1eshRLea0bBccY*1M-kH9> zzdCH9vP3yd&}reO7&q)hSCy;G5@^}jzFSz0&R6Hc7Ahh}f3>sa1Mvsh8xST%K;r>q z|8#OUdB;FkRrxoflxz)rBM_d1wCn##twmP$lAoX-#1yuVTP;R{XEe%$Oc*2R(rqjY zBN{oA6aI&IrWd+xNF+>q_dlaYi{6S!1aGWg%ojSost+V82KnG%wgS#_nLg;jCq;um{62y%tO78h}B)0&+hQx$-OD9_+{xrD&18c*yRuRJ+AL|xD?1p5o@vwWB{aVU$E^wX;WjT|@bW5+7gv+$k| zpFLf~4D)4XEADv8?rcfjL+~o#pNn|(pGBiH2I>L=HBh^a@nmG<--l=RmzlHCz2C*- z{-#nHmQhWxe79g_;5#l9xNx=95lT3h<5jO;Pp$M=Q*n>osel}R2QPD(WBYz6U1q^n zgX^*YSb-#C0kU%-5UFX)v~z{SHX1gHKG`_gU5`UlOCAU0sd3`=o;Q0?BIKIKdt$ndkN zcpoS30>`e=Y%cq$;9B=J$h^W{-+R4Hm8W=?-7QmoLRu~?A2KLbL!X;riMl4BRT>lI z&E*|}*h<9qrqk>A7i7+%lUcX2-i4%d=Of;|bDMgzFJ1VrG9sBwZDz;52V9s_0|(SdKuCg`Js7S_rOA}J z5Go{KvET03V_#JOSmG|NHq1h_QyW7yCp1+hG7}v+T}{ zB4vw({Gy#w{2BPCo|!OMq)drY-iDqQT9Rfs?4BN2Qm@=eVeoNd+nv%0#mP*b)Tasq zEp*?b!2{m{vFH%M2AB5zS9#(jNRC`Ysgd5LLk>?qj5SWEu{UL;f#iE0N(Q|>4ubn$AAqL<$}4Vy}U)nU-^(rO-0gMs8)%M*zGz0c0dEj z=L;!fj~rGYo_T;mA`u`}yw%ry_YvZ&Z=F9lULwtYGK|Y4e18v^uWD7Nqu$dt{D^I6 zj$Vuf*UuTBz&6Kwm#k2uDJy7}C+8DW|IqcSu>8@cyR55n1?Am6Qd^6Rn6cJ)ajoYO z2bJ6Ws2B#e5k($=LXh6!J;BJx71M08SEEuMQ_u4Kv;@%=M3Z635V{GWTO8=tBRa;p z060RD(R|l;LtGE#F4!6>5P(8b@ioUE!zgaw)&MBZNW_JvhMSsmWXwc7Mix{>p{adi0}GJy{tDNwgs4 z7iDuAC-`7}tJUk9`Ym-;#! z&-?pzeLRwx{H)K1WTBAchnr3;ejePv#xenqcf#wZ9io2H3}We=f5g(^g6VjwX7=-G zxN4^l`Ix?scN;Z^!nipUFXRV4sQ?&KcdQ&rb)JOkB~pSw@DRZHxk<3Dr z`|}GWhNQC_r1-V|LKH_@a*1EWV*Raxa?QNSI*(YF0X8`DaQVAmH<3t zI7Z|V;D<=@9&#FXHmUwU6}4>f!;^VB2aps#l9}2|CLo1HWbNR@p4g!4q&ovuosz2N z_){22D#MQ;1u38dC?a!D68Ap?&xhMkK#Vcn@xgkd^Meq|AbS%sH<@1QH#0B~044dE zCUk)zUpdJ~>?lq`vT~5*T3EY=MYe%W)ddOS;|v!xr2OcPbrkgw`1SLvesSo{RRBUa zoBBYG)Vl5|{nsghuQ{9PMKSsWf=d`nz+dV#8z(7Z0IPw%IrA?yqoYGc>DLHZc*@-$ zx9-3uxV!=&SmQ$sFvP0HPXi7CSB%(p)gg*P#f0s9ss>l_3{Ydh&%EpODv{mZgrfcD z{wJQ%(g5|@vOVP#O(O*at{78M+Ha8E1Jht7uSakygTM{$qXW{F-(J5qpW#U_f^CGn zWHjb$Xr+IFj*$g))maM1{D*Xrw7*)_;Jy*y)LW1b0HnUWmnqfw1F4XcLdvH`4>N#L<8zo$ZPkCf2VOTA2ndE~{kX?`%ouyw|&|j&tC9yE)i0Y7I!wBtl ziy&`p5pJoK!@*Vc!xU`4ye+8x2FGBvgFYb^b==W}i~UMx_7&TTCyz$~_CYyKv8Es~ zQDIxh4EbfCjJyL-f~erb)M9dS@^|_)zVd(O&XFa$fBV*qQ<^I7^Zv?-vQTeT%4S;6 zx^=)OPa(_JoKaM#P(;@>u9&GOH?o=uzv&^{Y@rE=k#(n|B3q&5oc(^|);mqep-(s_l_wHyxQ zIcS=iZ|#ekcZHI>ACMc+W7(hAI&U#*=r((q?m^zfzV!bW-0;mLZPw})+S*<#4Jnov zxhE7B`Ff0=rX|XrTdQBqg6Ner@xOy`cdt;welwb|+;cB&iSTr{?dkVDIX&cR)Ex@*Y3G>!Wp(!A&1*U!e^UnR6}-AG?4E>O z$}IAIVsu{jO6Nd7!EE$7mAq&K4+<$zsSKidfW0uY;Ct+^e5|iI37Xx9 zt;K!xFNRa6s6-!69L63|){zbImWooibP`AT=jO=X)|up)!x+i`V|^hN^Ken?Me_`| zZ^xcATvJ`iGwQHpXB8`~%kZ_(NieL_0X!c+laqh5pcX9sbEc6iCi7K=!E?w?t%DFe zMGUb~UN$osaEg?D{V)OlwUfRIcH0kgrn?rdjQ25CPBdy3TH>9@e@UJhx`J&ArUcp3PC5hHV>B&^TQK1e$Ne=-XpvLK(%gq}zWwOA$U zGhH=%bxx_GewSXS{J15yNu_a$dFyR_*1G0Y^uK}86dT`$F+UCzZg|_851CE^A3^Iu z@Rnevht+cMC+GlJz+Ulg#znI*U>}X$LaACtkp=(KR5&B-lLtLKdm}XTB853~mfbHx zfd^%1)s2`Y6u8!HNJwY9`y7dT_PD8Pdy67}u31EIzSZZQ$sh^|5DU5B z#FKyWSg!jL=TD+ zJB#T&=e24kw_2@+GiQ5su#&oL(vIYV&s4^b8X?O;c?9ZJU|5jMb}o`mfm-rAXdSPj z04n_rgiik_s|i0_bn#&LtLvNXuL*8G?m3_Ji!*O42ph~5`M?Plq(wMMy-v_`INct*HEFV93oCGNgvlR$pl!<(LMTw-! z9Rq+sS^yWIyeh*vilqJErR*h$bt8rMr^hG2Uhk)p5PJHM_U4K1o(<7#CQZF(?K=HRTW8B$&J$0mT#gHTocV+m4J z4+jp#n&EneuslAazVfmZ58x*BB2AjzTE@16g6=@#zRAiE0u<9p34_r%4_&OqsGSew zfFa+(je^Ad$J;MX%KQc@Kr#eP=6l_@FlTSRwetS1zG6BDECJTaM5W&(V#g|@pv0T0 zzf}1BeLd%0-N}*9;XsY}Aqwf3RK@u8U*6@C&!37|_3BtJ2fbkVA>_;4yP}h=9Z|Ri z{hQHw_sNulB^GBKqNM)g7%rSWFQUbemm-Fh$KN_BGyccsBI#Y$)zVB=A-sy*i8}J-UDtQ zery|16%s9>!VqHFZj)FFG;${)eGKjsy_4#d-d_JFKcUWVIMdqwduEt5!AaiGd$J=K z-#GcoR?#{~(Rd>BfuuCKw*2tpwb(0K=UXSz?;cmks6+ZcMtv0w*;=}(t@b;5p#v?Q zrhJ2Ui`4Zxb0)ai;4EZpG+^@M)cQYq&;O=M z)Y(j1#*6%(QuR4~PuYH5OYzLX-eFnIJ!fTmL}20wdIm#?Ue8j@4trW6oih1L+T$~p zw-&II^=c>Y?13)cG9WoMW-#B~(TW7(seD*BTSJUd-7B$awp*UZgidXhgFExx4=H@4 zp=@k^_{Su$Jh|-#cD5&%iiROl`WfD3pM_DWIb_0rba&a2|M)gFgofy0>RZH878x7@ zS$w#!X!VB{4P37S`@V$p${_>UPFf|6greb^mye<(reD^bB6c`jg~>WsZkS452}4O{S}=v{r}^41xi0F&ex+O z+FrQqD8)Z5GmY!OOa{_Uv958|6LLeH>DexUAGVOooSY8<7PVz(?Zf(d=Xr0Vt+N-U zvRUq*Lo@-&d?}Em5X4hqu$PnJ^wNo*=Sa@YEjMrKK3gBxa8lE32eAPj zzpi;LeAD@pN=%g1{Zn(^W`itwVn8W(kH`195|izP)x|~kO@M)MPVSN)YtENb_;OCg zJR<2op=D`p7foCBouA69^$!h=KxIJJv_W`Q!Nck1Xe+oj@)Vdy6ev+4#t;g(O zb0hq;?}?)b?~r2(s+_oDHIF-dbXS76gvjx?T=)C}31>)&226T~gfDGrI7|CD|WD9ULl8uwN*teCP7DNNz1pk(086?5RJa?ugA&K*u8q*L9j^8 zw@)x5*7lNe+#c3$d^t$Z!PW3z;DoHj{OY3GQ+MxaC#znux$eC6scuE=_X_Gnon82h z?4rM~fQUCSJoWs=VorPjH#arXB5hcFvr6-Wn6vw1vh!N>fP=5_Ah?O%{t)-6zafZO zTjJvqr;WHj*dBSWF0;wj1d-htL2DOy+2zDl^~Dn6R@d6*nf2*r(iF2VQhw?hNSuqs z-P7@@GL|p^a0vTxsn0d*;(YPjcoY?|T567xp1_WaNoXvoR(FEyIDFB9Xtu0h!43j;Ktz!`JUFg@?>;Wr0SpDUd?-vw1?AT8#U@MwRoDh!H327-sz6RGER%|&NM*{YUlqa5vT0+?BH)g zy<_Wx0kM6mgJMHZ3q*^H#T*Abs@87G<^VR5@?KmWx>1!t{H9rGOFG&hyS()XLpbjr zb(&Naf(DPi_lUZEIZQJGGhlpK3#)J3kw6U^MwE=zXMvr=~;?GI#eVJqQjARaz;k z+b9nvs`MAu9=P*2{`}8kxxzGxv{QZB>S@Y(L!aulEiAHn_)2DVT|=)Tr#2qDMi}mp zlG!>t@U#NiR`@fIISu*&VcIC>XF8-;_V*MQ;<+yC{i9C4FG^}J1{bOytr><6-2(wZ zLCP`0S~2>6lxM-2OkKC!X6k7-1b6+bt5{Ee@T=jz$N*$woqr@fib0a~)L@e4%du?i zmY{=nA}jOTETz>)gy+Kaem$PFtIuI)Y?@_t2-q(z7VF)LXuHgqF8?BT*G?+`jlf{? z?Y{rGpN;7(%o6lrSwPiouOz@Gls<_mXucS6j3zQ!47<6s_DRgbdS}*sIlCjBi0vwQ zO`4ZOTS?bCXJ}+XiUI<`>M!ZeFeP<&(JaEh6w-uocrDHU^>HVRydHiiu-=kAKta8m zV1l>+Je>vW45r@b!17`-v+*X3hd=eZ>4%}{KI23&1Pnh#0*GVqgYw2ZUr7IxF(8xS zG}CT1w2IblWTt6Z6T*WL=B9%&1q~*7JnS{^mmK|P3Qq6R0_N_?nha4O9o|R}jIY2t z+-LsUY$5KmvBIE`bu6U&(m8fXgJfh;F_5nIvSxZHF%+)7Vk4s)_30X9Diy*^nqPDR zecxo5mGAf@{{;8AV~F>+Mh$ItpI*$;n@U3*>1Qdl<4kxt8vtmbgI$$Wg4Y~AJp8oo zzp19_eqk8^-X_DxMven$fF>^)geC;BGO+K`chin*7a`IL>w=+{$g>Im|Ld>NvxY2a z@60h@h0hyr72wl?>ZMal)z8aJcM8im5!44?%~i1Jm5>td)s2O#6-Sp&l+m!#wt+JV zzba*2tR}ANQ%ch4&ohS3I3LA=MxD~>ArYAPY{EcR z4U0G6K`=phY?wSENTAHSNqFj`5GLk0lMd-VXl|mmAFU=Y^!e-m8|%?jQ3y!{IQzSP z*PV17ai_Ij4 zB7gipY`qCM)a(EMuR7YJRUw?Tk#K};SvrMKwk%~VA!Oh8txY=CC`(yJAt5Bo*r#k$ z*6cJG5weW2jcqW$*R49A&*%I9pX+jUonp+q-}n2zU(56Pc(rcB2#$ZlOuR9crbuf- zzAU}fXU=v#M`bO!Op!;n$F*f57tpfmlQ)viB*7}Zd6s1l=Y`9W`7X;eI9cJXazh~) zsVYD0;4{RHE5Pp8N0EjzPGH`fjlYWVdiB2rLxfR<#bhk3dCrBCE3pxV@)z2 z)hHg(^`9sYrYK6~xvsxje3D&Ih?NZV!r}>{(Iz?4x%Rd>1n?bx(kmS-IR!g7B-$sG zuDuXxNl|j201(6V-jLJt5ri=eDl^B3&Nt6|^RUa^G+gS{)fv|Q>gm${Y4ZLNX}JrpFj?oa zPO3kLRq@xV-+-L$vU#u@hE47TTL;f^;k+y5k-ZBb*3&7z&D*b6q7P4mGco2#>}?xw>nR-4anlypcErt@a~y-=OAVDyiTN$OtsFe?-p~YB7C@& zdqc#cscoeTA8oz--mI3{@<4593uW89F7R}^)Y4#=e3rJVmdpifZBgf~8;$W6JagtU z3q>zPEHxM4f@h#Y!^S}f^CUn2O2x~N27S#@ICxK?GCXH5!Te`%;<7c4(GooN$LFR& z)Z|4^2;kgO4t}Yo?l7=5K#C6%oX^U|&7f?6Bp#_c|8P957R=WsCKrws$@-+fAn-OCxo6%@PA?lQybc560wjA13_Viw|nB#CzrUkHfgAgj@VC!z|_ntH2Wd+4A}A>Y_V-nKVTib3Oe$K1$=#*KWs5sv4VmmlE40|c?R#P z`9re@WH;&A0kE@zgzf9I*u|HfpbHO~n_G;ESxuVE6eRGyA{Xpj!sBT?N1ynur!u0x zX!D%pxpRGg;{v+SbBk<6U066O3`$*`tdGQyoyJ;jI)@Tqt)ntKCT+0NAkm^ZRB#59 zK~ViP(Rmbv^$4H2i6t0PFeKvEn{D#0mCn&$6Wa9cOarM|&DH9;f&6gZ6793zijUB( zGb{RArg^xz|DL1g$~DUz&SAHoOol78A;^h&$;2Tw{xNWHzUlq&3wk)?I1wlK?bh!L zX7vVc`ayQPs1#TRsEKAM$kMU1h#pkpgy+8S5&r&lzp#dRk!kfilW=Ar1|0Xm{56k> zbc?VJrD0Gd#S&}@oXymOOtATeOy{4VQrt~*yMmE>ZfNj+d5f>NcS4#*@*~8X;ynJR zO(cB4wP&bTB-@dd?6hNcZ8zr!$J9d2PUD8Dzv$ahqnC?Yo^Ab=(yJ$5jsMFqa=wLNE$Da( z-FG>8B=jQpm#aX&(E9!JBAC1xKHP@bb0t3~=~ihUf5Z6aZXx$O;P`+DF$3I5JfN!> z>IS6M;#YFvs{$t5o0BuJyVbfWD7XPoP1=HC>{#xNW7m}GrT%<3cPdpQptiV<1t{BFaq?1c5!TGT6()yj%H&`d(=P|!-58o>kE zq7A}Z-N`Taf;i57GF|E-O0t}WeOHJMlKb*(>yO@lcRVH6X%F6;pM!yd&WJl z$vrP!o1%XncBJ35FytxbF5T|`e&Q8j4f2kt=!oRKm@%pK5+^6>Ld@X;{?s47<)y=~ z#ycZQ0_LPNV#ED9(Whi!_^a_-Rp6n;FAuzCDE?s3fA%y7uiOd%oFnvj8Q;3eNOi_v z4WWaD?xc?@Y}ok7aq10J)B=}%VGihUUiaI>|MJ;P!N;V|;A19WL*N6mhEy~Nl;M_^ zPzzg-JuCPv<|84XYquz7gzi+%DTU=f$@!_1*}_R#XowzK9-En0^S$Z1zrb321=Q>` z>Ub%ORI0HquX+6(tWW7#!)StoLxr@gRGwzJ1SJM>=ukcBOos>N$s(PgUq14b z`(ii3`)*+0(s0Wf_{l!_iCE$b0&O!l2`oRcTt}X={7IAxjt>B>h`^e&yrU8&A+Mg{#4w{h|0Xkwkoz!2V1x)14F<)nJ%~n_2-VP1Aq49ob?M^kJzMqVW1d88BpEy ze#x85erB=lNz*31_qli8bQyN3Y4H2d?%sulvc11&Zg=)aM%O*bo;xxgwaO1bXampY zHz+?QAtC(MJ#0{as{0YdH?ABBWO4u_++85|r55LLb>yivpXm@!YfGPOB>Kw>Kt@k&`vx^vszXeL`fD;ugkGlEH%J5uQY#J@JpB^ zTEBnqpKxUxb_d4OX;{6$QuoHDPY$0PM?!$)1LJv^cC={o>wVUFlW7m|tH>0fGKwy6 zs9cLU!Kjz@zGu=f2OIHCJ-W>@KE_Dx+uTkSDMx)-x{q>NYwHQ6tg5d(DL0*Qa*p4M z@7uVA-y5ndI3y!TU&?JixHela7&$A+;hmY~kg#om+%kjpQwKQZnX<+L!%NNZHm{;D z;Ha?y!_to9{HsrT-h)?UlT+=>v(Xo@f!d~tiW4i@ePd_QX}^jM3EACaa~>Ws^&P0Y z`(etz(^td9odhI;pZV&Az&}Ru@72e`%1Hdx(EbL)+{t0afFbh>$CQvpQKuYVJ#qBV zFz^>TpeZpia}86-M$FPP9G)u!6Xe?TkeZdFwsTMUAHmRHL4H)bax#u89pR=4$t%wX z6Gv~(iW~G)kX|l$m4lJl*;`-yR>*j&b*wyUV0v=N(kp)NFRn8eNesc*6~#{Il{vdE z-44rsPy6hbYJ`jn2WXu-q(T8(@gJ>VA!qB!iq5vTDvrOkV zHKR=*p(<3sgAh9->>cINY@NyWd>LE)s(#lI-%UH9C_pjns8H!aeFgoVo)tGDOO}s+ zv729MftIN>W016j@5-~XlAqxFsO19YB})q=4<`&VT{_=WR|h=PO;CtOB>plQlasm*J;27yWfCCQHwOE&&=6j?BlAWmGrK(5a ztrUBX5h{|0e;6JNcMsE;c=s2T+@{;IMmxZ=)#@&X)FKaY_{{z~U$8zRy|$*II|*z@ zWsmf`Hay>+c;+Ku^#dRKcee{a)flkmiDcwbsLh20A_ElVsGb43)<%CbMSZ$8-<9Xz+^)HB`tnUYd-86W=$2ZywY(lN>|mho?j7Nul& z$%L;fRNWT(8$fM->Z6Mcu_skKd=-K;a2x6-k*%zLcAa3|wg7?RjI%7!O4gxaEl!Wk= zX!XEbJ|y7|6w`^tmOcbF4c6y_73jOvqj0TQP!(2_9YH6@M;E;opcz}?9XOYQG*Rx_ zboWsB#QL^%Eb5E(q$aV6M@GH^K7fjUnRCeNYjGI}lm$aIJ0{;VJo_7*kfu_QGeEKG zgc6j4-WzxxD+A*$P;o_@*rkkvkaa2!6~2&jgdy_;lIcb`PleOSTN?aBOVvQ4O_We; zh_~D5j3R`Urt`Zsoc&fWNUrVXVz${Yin&!D974GW#sen~5+-NN&^9i=vQMj;hPZ+I z6Ne#xC3}DpyS5z6-jMim#h}trJk;Nja2!m$W|Oq4PsBPigIn3K)p!}OCoS;%gtH_z zM?)731#}Gp^hl^KkB>1o$DBFpV<7A=EpzsX!(SOkp?r*%$z8TorKhLZ#2udWm?b@H zN^C5fS31)I7p?6ySJv#;D7SeoLZ%vh6PyKh1YrK@Rosq%%iA?nhIpj1Vf5gyg`sD1 zi8ddfWC5p^JNc_hBpmF+ELu&J5=>F%G!O^J3yF&89`XCyhlZuTK4S17wM?~w|5DY# zI|W{4ejgp2fPiUrbid|ciSOz+W!x8AztcmN1_6f~g1ex}4rf@>j;B+unG*vaeHGjX zRz>-jF6`z!UwNYx*MDQcZEdxwl9iQ7+J}DIwAY%xREn_}yy?H@U%T+`Te3^DFEeY` z3UN}g0Ij|hEYONLMQU$u-h8klI~n$+L$JECXtzV7>2v%znnC%jL6+fi_6!><(2|uR zVZ7&;Z|-+~OiSh4Qv`gU#||v8Nklr`*fmmP{jto$74|mHSedpgK4({*K2#A*TBTAb z7{$v;NymA4y{J*T^W%IusHHr%BG8qohONlJ;IeyCH}TYVd^5HSpOflSZUv(s>#=SS z4rSx7dLp4c>^TTEtZDd|dSzS#;=Yo`15irAspp=Mo~x`!nrTNUDDZ2BgJS!p@L)*N zB-DY%*4$Z282?G8Eu1ge3kxR0!lj?W0R=?Vm5IscM4$##<}{i2fQ@b()X)T4oaphU zQ3>!vf%(4j1`(>s#aY-BxTb1N!|f~U-hHK~x7|4F1zv7xN$YHQTgab`>px%8EjctQ zqjBP4=s17bKk|DfmaW_TLDoye34to*S@iSd2@D4MfuqEvGq<6$u{JcfE0s+cI^%w% zV{$C?ad-Tie>1!fB2aT*`7u;r9;0EESTe`NE~6}_Kqv~(-9^kL$8mof7~EWQ^0|{y z{JxaN#b#W;L!7tWYCW7GZvl1tveD=y>!v~2ea4#{{TGg$w`FD1G1*RyAdGmB9rRmE z5YG_~G|-)QzWJi2uCpNEH8rD_XIAdmVpU6gE9Pq;p35kua$r~Vs^js*1 z?B^pEc|D>udw)bBy;3)EHQsHo==> zuf)UdBHRR?Lh7dnZDoempvKQyn|DSx3sw_ycQguox#{uCS8?1%KM|aocfdJ>BqMNF z!$0hsW*;M#nw|T8^XtmC4J9he7|meW0x?ys}asA*Y7T2t^S|2p)uQ$ zu3h1r)9sb3W$<7Eld9BGFvb>gCUirz1>2Fsj@P}1(>84h{4}20mMw;2gUNJ@;5`Qq zr$H1DzNzhg--An-zzGQa?Fg8FN35eTbuM6ED}NTmi+tMR;>WAj1X~fb6i~mx4VDjM zy4Do??0%dCzIdUm<=Uu%qLGr`^12gF^fPnQ`f47{SIoNq)hFuCI_AFfR^K^S3?MA} zTI!uPRk29th3K}&W6tDCwp18W;_9AUI##3Q2GX?S!A6>u4^_?s~>KLP=Ny>$^`gO$F7VE zamYrz-jKcO6_hD=88{b6y51ShH6=r(#mc1*NHYAnnPi}?SrZo$YE6L!bi@>@;ZW&B zCY=SYRaGEUi~|=1oXY&va^n5_K`vW3W!>`E|C&9IOyBT$j{za4BV$;5ytOzvn~8Z& z&%5+ZVXc0m6|#+n>r^CzU;CtQ!AL9dV*pFk`> zm#2Ef|Fh^@*(X_QZbt1lE!Go*$JxwVuy$`tZz%u){7T(uU=;ecXM| z%9n?h@JBWoE3U9|D9OC-{ALSS-M-B&@JbdoSY;LGxFNXdD=z%NEPb?xYgmT}&}gex ztb$VAu46F^5>mk3k2{~;)rvC|?08Z9K95Q+Dk=-!kF{I`@k1~;ai!a&Z*P>wr z{M#up7u~B3*D}Bb0%t=^`7aBu+*Q*4W?Qz_>g~Bup`C17t!ThJ0Y~*@ER>?5)F18D zUkPs2aOWFY_pIRlL3kq5tnFYB^c3G2!~MZTC|fsLl&3LP#lTeS?er-YC@JP#qc?Ug z+mLE~e7jTM)fQWIRT{1PneA#z#K&g3HMv8>a4#?bLtJMoj!Dnhr`=p z3lMLQgvUTw38AIeKi^vTa>`oCGZD|K6dYW;r!`81Bx^^*DkC(=!IONw{N0CvOn!5- zGTpk;i>lI9lj@*IiUj0#U$-6cd>y$v(SVfU>cLL#qh=A;*_B!K1n$(2geq7JC|#e@ zoy*kom|Vb2OL(u5-OC?UCa9)Wte;V$@M9*0daar51`49zZ$-|6jGaJ8W!BBn@mgU` zea+diJa;q5f1NR}zS8Wh?AcXxcw00lT9ICEF*XLd=%1h#riSnSQg{${ySc0(JCJ&= z`3NYMXYWu?&`V|Ruv_mRrgo=!zpp>I7fB*(I)kBva;Brm+dd=9jem*uhZff_n!*RO zp}b}ppj!mGfMaHDRF*#D(Ing~lLo!8aDw3~jT9BT6wtLC%uf5bjqA=>WKDNkK@=jRNC$CzF%Ba5 z?n@#yQZQ7VMaYX#`=Rka0wmr^r=#gO1AHH;I z6Y+&m{TlUmFjXMsL|PZK{$zVR_+g^&j)Tmc=l{kru^)?Tv@65$jr)510`*UHPfwrs zf{AX*T$n@s^i&Q$B`)8x&WYUT;W8OeBhfC-)t+fijjUz*=?oj1=|(z3@PccMnP!e- zQ)@z_GDf`9Q*t|I%3XM6F)u_5U-egY%;2Aj+S*ijr}ZMGX0&BkWbJ0{8ps9mE!$32 zf@bxZMuMK-xMQe1DdAJg%OMGwP>G3WGCc`dAaputGAGzUja2i**Mi^{qU?;sj>Ul) z35#sQ)%h|%rzIAL=#&vRS!oBA1Wki&Z+8BAZoub4?Q++cw5^JYjDL$v3UnGN$2{ht zRUJgSzoG;bRhqubyEU)~VgGd;yO&ZVV5|^T~GVBCgX|UWyj&#t$z$ zbAbemdMC*Qw#7hWg~cCeaEAAk_Z65#fWfaT!;b$9V=;EWVE56ZnQ%5A+#v+h1H?vnqY;-I?_0<0uLaXFc&xoKae;powA9XNq!Bl~W`Nt``ar9!3|>9MJ2dRw zoy$1A_e%$u6sSWnad8vg2RWy2frHpa&;h4!%Y>hmk{*}~K|LT#Z~yV?EUNLE{^5K&lzo`x)%tW^ zsKTxDN=B^=7R!ozE|cpm`J}V{dYX z1NGLnz*=K=o43A1@k;ZQjcatzcqdjQ6N8`_aOQS$q?-!@d@DAX= z*AhNU?+pFP+M?E-`pm0%x`mrW%v;#T3z+PaWqX%T{~ARR%6-pB?d+yaL}ib%!d_* z-kqZjoMiGP`_UHqh$N%e+_I!2cjGhYf|sAI&{1&d#d+slwaRt7cIUXr+QW6T2mviAwN2-h$%C+1$)L&2k>*1{63} z9j=vz0HDZ~-nclalEVg<@|*4aCsXH;IaFyPSSz<;*U!~-b2<^P^f{9*aZU)RK7qta z@WvZ76M%y!yUA*N0|7aF{MdD1U_51Izr@yLbjeE1pkShzv60f47e^bjfjS5#T8K2hF zt^R5d%v;YaUN2=~DX=!?byNk@ z$ahKoEel)mbJYCb*H?%U-v8qU*JXv+Ylw~X%!nl*uJ-(YvX8;ACvsd};umIOmn z`Y~(I0dap$_^8u5#owyuOg`Y%e~zzNjLhcpZR0=bc616J;pD>nA=%GfHHToPK|>mC zRtGBhhGWQxK+4L zjVhJ0^XeVHCF~4wjkQKho8D-kVGwD=jB2X_FY-EOU)|S62`E)lNGB?sbIM~^Xbf*!zo=9UF8Z(l)Sulmpf-x_qDJh7%tmy zrcny?A^lsN&mlXznC^YEQUT+a%Lk>YIdDLQODKJ&F(x*Z4Q z*f%f>ug6Wi@qCBx!7*!{*|0tV!fzV-d6uB>UXh3H-G;qnDAf17x*JdwCwfGi*ekM6yAcm=ee@E>Em zgtG#&8wp!9L7R2s?-)+NU&sV(a{8RhLpp_)pX|Dx(3501$0SWoV zLueiS=K>`M_Y{A1K1(3SXWm~ES?iRvBKAU>>E{GyZ7jRsI5tjzZB+kL6D>`ZAAE!_ z4jN#f!O>75{Xy%exAx~R1dXl0sZ$Eg0OC2-=Zk|4sOS&q(;+Vw$W*-22_x>&1~KnG zP7>u9zZQFk6V@IClG1U}0x@mRwKJBXLf?)UDnDVgpi8g22WwF>KmP}~0L&zvgOAdk z@p?gJb=>~=GSv^r?=W#4bL`?&!ac#;DqW&;T|-ryVdw~`vwpuH`17xe(3JsC1@ds# z-T4JO1Cy&#B+xyUM4F%@02n_J5J*9t7QC>f?tD&$;8q1T#z`evPT6dLc(-q$(MF25 z*up<_Sj)74U%0?@l(UC?(Td}%GW^VSDEsexR}{!NH(`IcM+a#g;8&rB6wrf}pjV)K zn`j;CxGb6jg+Ay*0o@TH^qnIlEE+vkBSn0=vk)W?1ZwT;4@d~7;yTrC2*TV9e>{76 z|0VFTA(lwGVmQLGfVV6e)W3-Te~^QE$YLBXtJ7AE4E4elON+`NX^WkNCmw-PK{Pr{G#O{3I7emJX z+P8omLX#w*&ZN{T+>AdP$N%ZZ+X?gxv{XQb_o z!|VM~x_H6XAr)JKn>p%y)ZLU{dai|WD3`ob_8Pl72JYdee}NAN$RxZ`_=O7wpxAGZ zPs1X-Uew0GaFCX%Ex+soeCdbnzRImY{WvIfTRqoQ4OR>^m0yBhjFpCZ%z?4Dc9wy&_2P(R3&M-m``h?JPd)-uOB4#%ttJa1USMfiCL7l(fL*XZTIxss;h4);#sZ-pjK zp^?YpbW~Ra{euhU>F>!GKt#O*!nR0cuP-r3zU7NnR?RFfr0eo}0mx-tN8dznUg}$h z^5h$U_;k$)@~gu~btI#a8MnVQ$^?dJh<3a$3iQq@IH?9T-^|@)D?iF@ z`Zwe(fh{x^Xcq^5I_U0>f4p7l<nZ(n*~fE>&L!TT4c)cbizCclwk`HaT`1c4BW9?J__)pLUVI3Z51(ViX0M5!n01Mg)5s0YHeb zUi^SM0?preAG3}%5S(gd86S-H8^*mUd2w*(+B&%Ks!lr}^we${oed~B+{@`LxvePv z-#RMP;554o+!M0PW|emi=OnPyc2tqck1b zQ|^E#x>j!SYTxuxpbr6ZSh%FIA2tpc(F0QiT8e6+wUrKC3sYpb%(PgvLq-!sZ^)y_ zMU>1{;g?@_0x@9z)1|hMy*9AUWY!m*l1HFo4n*dLKA2=;s{c)5m%F4P=xdWzrU8fx zATHQP8YVbb$Nz@z@d#mkCCRZFilJvpjO?YCriaiTX^1K6HkIpcD_6`f@7%uOXMO{Y ziF5b+87Wfqe%*5XWtdfx3cHTA>{@RTsWMW=$h5sD*;`lCviAxIz=b5m0s{kJu$U%{ z?x>B0yU}OWVDS@rj`Xw(FX}1xkHS_3c>VDi(f^*s;n4_{el~+d#8QSqp(8>+LNdlO zZqd1E{c9_))+MXeblaE7-5~CmIIWmX^wbrb5p?t$aH^Er$dT`EFkqHOjnpAb1GC_- zljmw4Ai%)rD_r>ziRztAY=*Hh|5rCa98lUP#oqch^=IeWHO~X+GwQp^}5(p(nWy&si^vRTa6-f(-tNDE? zNHV{6>BMkAAP4Jn&e2Jjh(jELqVw9W_V>U&Rcak;lV#sDn%Gx*sCX3q9wvQ!F6sva zv3&OMSU-qol1m-e*_6;p*xQ-+4!)zscWzuC)Zd&G+sl(rxLpgaI$nJ$kCKN4+~~ay zqKfrsQ0jui88%ti_VwUcpAZK{7TTW>kPEyosP!%>NZ8G0IO3(gY~#AtF3~RCS+wjO zvlTzkzUn$kb!+up`^H6YdmA;{*Oc2uy)M4V_`@Esy#3et&ENq{stMqnDs<4w(ynj$J=E~4vzaGQ|Hjk^nkk&t+kuKPSpvMuACCSo7D0ZB2F9C{!_qm>o~iqE zmd(M=QCnEaX<}Vp{bQ2zM^^zgP#2+HdAuz%5FIK7ENZU#0N4Z)-8@X1Qo!BT5@cq2 zn=F(g9z1DOE=`<>g?VjeS!4^hF}Wx(82tCgyQ>I2KhVU!0?rD8^X@O*rsDan4c0mk z!Zjz!?!3U`T7&h#c`1+a(C$31@vZvta1NDo)-%^lbr+|7$^{WP3%gx;#?3s;eNR~rQo(huot#RAv6#%%v_!|%7noxInP!!Doje-TRX0;QmgUJ^p1iJ6{pTh%4~w6l z;e$wHoLCn!jqPC*DVqPxpvAu7qWn{dpcelHgzd2gTYXs#AVs^&pmh17)?E{kp92$G z;jBA%KMtN>v|}PH-X6QHH_nc?lxKRmP{vt$&+^_ry1%u%TPDKoy_Ug4Ac16n=vSIk z3R-rN?Zww=$~VQ-5`s65!q#u@5mIj(^K-N>tk#=hNZ|AD{u{Gtz`@D%$3pGnczcA&0)5%?rJ#- z!*8@l)%~eOr{sP%_w3!C%SFlmUY4(Odu-SwUEcgBR|D=?E@1k8*;{@s5rhtu+~N+P zxVEX~JVMiliiaU|Xk3D&!m525#~cY(HJuY@LDI?VxTc`wd9eph4iqiItv9?>40b+@ zNB+CjX$N%ZJ|jCF+CefmGh-z#8~=dqJJc5z?EI0vh%RbZfLMckI>WYH(kqt&T$~p= z(0lk5hjovt6p6v-hN)$DK1)$p6CQze1|H@+Wzp)+H>_#t{#pzM_wSwiQ)YuEB)5r` zmGvv$AAj!LxrTqY>x_8l#UbzsI?&L37VyX*=zeNFNzhH-@6%Fp6y*m3(!=|Y-1yBv z@Yghq6_C5eMffC9C!~z>I|%8W8lu&J0Q+IN;-Oa!IfpVIfk({0$C6SSl|8*tKLeCh$o}C$sF+6@#2T=bTOJHGQJv{$q z?fMBNgtUZB>DMp{#Jfa~-U&;10}6XZ;8{L?%X`8$8BN@SM-$D`1Llq@#EVLi5TL8< zPK-|5@a{D^O9I9CjTr>Xk+b)!m#H9J*)ZBP51qc&2KN`11Tx{)^!vrM7%u`=ILPIs zK-Ow@edIixo-U+I;cORgL7J@dcI6791_BSd&WvN{))!cR%X_(;kpjHEQs8ny2Njx$ z0rxkiOn^cCK#Ti-S6CS2p8xUhbE-Fw$+244T2cdCv|%=Fho1hsJBMvOY&4(2SZfgac9aRKS}c`Xbu=A;3}y;S^N*`m9Jt z<*)!xTRa>*2R-{37=nHwqw94?Qo=#Ja<&S8FgM1j{)~#%W zEn9-!%Cn4e5))%342<5=HPH+?wCsSKSRuBcHTmQsa*g)R0f!Q@rhWtyNAvmMSc5y> zA(6mM{ z9zIqOpzQ$j6|_;Ub!W2}8*Lm#xR} z=ElEC(hJo5LkVgy$&dISAj*@J*w`b|KyPAOOq2cTgjOH=Bi7*e%|U+_t+r=jbU`@v zleOD^<=2byB&lI#tlD0O{u~zY>qqWd-2pbhhQZqmnDNfPzxjiC9=L}=ZT{=~l3qNA zJ!@k)D$Jd|5(#LhU;py&A9Bkt|EC-R|L`UPDlN}Wz|Dn#`3%kzNIqt_&6OicMU%^Mzix^4 zXI&%T)U-q&O(S1w)Nd?k`yDCwsrE`S(iiaTZRA!mH{gKV4Q_xD^0WSoFs^7EB44#pwNOyC_z5xwkdtYAi9LHC?))=dCJ0*wAgbR2k0 zvQ3(se_D-^nh1(RXm%PZ9DznY71H05>IrxgXAsvE2JNA;j|-`P#h|MR`NBbof}L`N z17(G>`1xxoJqh-VWbm%Ezw}MXE*-J@izxc*y{RePUuixH#m-P5q&^xob2ag}~?A^(8Dj+iUq2HcE$4_3vE59(ddL_H( zTI-dWeM-wFCVvJj-(^0*a*UhXs`LKRACD^qg7`b_R@alH6Y|t1D+Th78s<#QmNZES zVfF591kAK314Mt%#6>;e|y&a!pe=ymv>8yRr3qD z_skx0Y>Cay&3?l(M3rc6xF*f{CC6TWIawuR_t?7Sda%$)P^Pe9oQ$c?Rjd%7-pDB7 znfIyArS()ZGe=_1RM?oXxmj5S{P-c8sV@}1 z3`v_rKi3DFt}tJfw(om=t^6!5CFMlN^BcM*@U+3mzzTNgX#T~`9X8d$H|E4}m;VR6 zUVcM!xU0PU%zU1I$jeI=8pA$0_!;vvlSJHNORmjwkX1~v6?L2A$fuU{(Jd>~P(x|& z%Yh4I6Zl*+AMyiD;ZLE_=fyLfK{(U8zURgyvwGdz4<9tMIzq_VDay^gzOZQKSa0iI z{d_be?1jElkDBJif#~JsnFL|gVCgu~;Kz@Nc6h1ILx*bK5AzK+I4htFuyaS=rHyjK zT@mZ9?9j)Ln`LNc$24!Hsq5<>bslXB1)-2UrN8{C!0F6q&z@CdM;k+^>sS^L3JVeoWi)FMhr6ghRRpx|Dif*BKRRMMJ%SjI8Q%=d}3ho2$nN?T~vd%D6x-TMtSG96EzJUR2@ICIUL zeN`syH>*tQ%AZfy z_0p*R);EpI3f{i^YP)aW%&f{TlS#9~HC-#z5Tmuh*B+~j$6S&!NG%z!f6DGxoY?; zS8HqO?6>(M{T`;44J+iTncg-vEh!RQg`tMFb((DJ={_-jnKi|wAzZs73uI-&AUI0;A{>h zD;OPJ3zK?T$HwLR{blS@@C<8Cf=}JMBcdy>^m4j}b;B~9mNRv;LcA&Qv0oL?OKxGB z+{+>5`cDqlML8pxLhV{DYIQ67_BGeBs=)L(_uyw)fvcc$Xr41^dg}j1gl-(T__!&M zlZ7NR^wvs>vDxRgUITm`?XLJ!b+z`1_dH}*mKn(`&)^`@VKu|%c_u9?FdhS&UWM&Kx*IFOr$E=LAHttE zwVp49X8d7Z6T_d4AaYw4U-^+dl=PrtztKu-Or*fJBt5g#)S1@cxuHZQ{M&=CL%WZ@ zS*!hg$@IgAi&3w$3=7VUTo5Y#DmQbFylXICgA`uAXlF3&O($(x)Mn)o(8HZycc1?h z>q5#$PkNN8?6QaNnWOqG_hp-db7Ks&hKu0N4?VIb)YdJMLWLFL+aNO@`&^?Qe#&%Eu$EA56cS#=MA=7oUU27Q^jqvNbqy(X z{5uu@bTrYAJJEaPpLWvcE;UXz*vssxg_{}fmMqojR_q^5@b|m09w>VPx_Ih}Bz6hz zT75^?29Nn%@wxdnZg@~c$@eUo6)#_gYry9^l=b-@c_GKq3s+T=QoW?6< z(*~R63i|bQZt(KF$Sz5`qokx%R~3FpQ*@y{lPTh_zh zzgPcrnkndj+R=RSt}P6OnN>{Wqx<&3>K~+v^@j)~F#YrRv392H71thV+aI4X`c1p$ za8q<*l>4+nx!1L<5&PB@rKlYH-XbBY2Z|dSsYBKU7rG;U#;zZjZo|fWraYSTUVAqk zo0n`)AIcmWnFz&gSUbL$^pLqw77J~b!GuLZ)I2g%xjC#&ILOX z1V|@w)o@OxJ-$*s2_Ms0p9Y48Pmf66c0d6W=ksNnmfRAv znoPorG{U~VNPz>}f94I{#%S+cvchj_Nq6p&49&cjRCFqkqyPA!e%LJZr03sFv3~4& zRbZWIun?YIlHHmQCt-w!(aP*`?yZU9;)l!=mOtfd-Eg1EQ!{JmTI}AGP5{o{Oo!i= z!ZUwfpsO`oTo9zpVx}f3VXCfMW#0K-%!(=PGRzWA3@@L2b;Nz>Aqp*6`~0krOHb3# zRuw_Inx#$n9^cyw@Hy*VkNOxTXPJS(%e4x#uAqc|7(0ak* zzh-A&REH>XL8RoY5YJgGA%W3X7!BJ-h_@K&TrSU?*sh;2ullir6$O^U3xPz-iH{4v zR1Z>baJpy1A3ugCM$(lY`grO4({S91uN0Tpjnu9Rrz>fX71wF2uw_O4U9m5&3a`x= zSYJA$U^8{!dsVlz>&B&J$+SSdIZR^G3~Z+0{F?lW>Swjq(c^V!h%?uRJebc7cPQ z`jvq7(Ij{KuO_F|b+wTMeV3nHbGuf8Ub{P@Y~ic*jQ@46kjlzRTcaa;(UCFuh~1y% zc#|aGn5Nqmvr>`h~xivv~vrGCI(Pk3y6|%!BY?jpL7M&`|)_O%G3rVU{dv zwGMO{*UZcfVQJVk)5|tY%r3ibV?jyx7#DZf8aw0G8TtY)dNVJxbDXgG#tBgOC|Bs`h%f&=cta`#)0Nq6Cl-G7Wi$)g z_sz}$!F&%plzwLj6Q--z#KcN+!>WiDHm1_AJ4D*ZMw_-em^BJT_iyyH-CI4yvZGsFL0m2%gW01 z1Z7W)iAlOnv7yrvg_e5#kceGt6t*1XPrp0es(YC5mLwCO8`}EcCs;^Vy|RkwRG#zT z%WbE1*?BnEWi1p*>jcH%{4Z1~O4$Ma*|}s%;(IQI6L>+}&?r7qIDVa)Bh`L`@Qo~7 zn9JE_#a7Bcnz~2AO~tJ`H`GC(A7kuFlUtV_KL+o#m;jq_VeKNt-3D7kUMMv+(9l3fITRmR zE^TEFg5L*w)Y*PNS|3WG)0t|^y%zQ|nFqeF2b0D5lLmiC6`mfwaJWlfRz{%Mi zESK{oxivL4GwU>puGE@}(Te6|A<>TLJ9)TbI>5^RO4bx5S125rY*_ulZUK8$ZHZu1 zc-EBto3c!snhy|84v-J&YLko|#xjG4!kZh?-)Bxf3pC$26@R9PTP?17&;K)e{a!bm zB}FC;0jw8?Ney0S4v9;zXZGQ4H0Ss^URT&NOi3^tebJFQ^kIQ90!QFXx9l>S8-lR7 z#*z7I1z$4J>Ho3!-f>NC+uA7VQkP}PLZvqaD+mZkk**>LC>E+z1(gnl-oa%j8jGq6J z5I*oc_=+!ucw>RE*5ZNtS_DM$4TXaxa)Velp1Bj>9DYhPYVP7leEz=p9_Lzw?+vhPHX2ewwX_NB+s^BHoLNYalNnQOyUarR6fud8%Sq##@N;&7s1Ug$xRm+3i zY+^%Yu+I1g+(rt!fOWM)(wAMau1Ah0h+eyg^+W}yepW|t@N zGBMYivi0Wyeu(X5jE($T?>?ShoSp~@CxXzn~u{wCJ&;sVNAcgHW zbKme&sgKyA(UZZM5m}Z4%Scf-Ta3Y`Gp*hCi8y+cK9EUZn&0W&CHh+J+u&@ANHw-3 z-oL9@9v3XLf1i(V39bV@cxl1%J)j@Ilr}i^?tV(zU}l-eEl2Tmo9(@RRfZ2zUx2rD zF;Y%i>#}0`ZRY4`Qq(}A%qjjk@XZr!tD99O8$4IVnWH#v>;(nZQ$lD%^QzSi*Vi$G=qvXpYl2b^qDoaNVVIrYT%JtXoI7oGeuznYr^ zAS$}YGd)Q3mDT(RTx%7?~J^W0_3+ueuZ&B2%>f2Ix0R= zF#Tv3?qveFq3JY7gq*V#8m61uCG?xIsoooYe(yUemr%0E*L?z;*iPb4bA)bs3p}+}QcU&0#M4rY2w{gJj@cKiITD;&}?m1Jh zOllizuaB#Ie)ccv7*HQWFmi#~R`70}j3(}J?i^`)A}J&!v9KrX?qd zSJ#3x3SbGCqfHj<<3q?PKo_^#kO%5px)IuuCA2qmw`k`?t-t%mldr-|{rQw_jedV6 z1P;|?@wQBkA54S{OJAv`&rgY;i(^ew%(p};Ri}*w64u0VJQ9IRl+t~D{tH>G~AIo zp0OgiwFRPX=1-x28A!rE=OO<)x1tU4fd57U5xrMnqLo4(z#xC8;V|8fOvJ)KM|Ucq zi?(ZT3^r$(RxWf;njJOVmKpi?9BpvyqYrO94EhbD_?Jxk^?#w!-v{g=OY4gT9CW^VZopv9%M)u@=v-gq zMvkvOc<>)muVoHlL5_|bD@Da0-Yd!R)tl(rVjDhwM9>)`{4d`n9WmUqWAC3E>G=1! zsoo@I#c}~`H$$qn&ojr7VD&(Dbsu1w(fhsNH}0#=*xNdyQ8u34VSH6GLjT z?_Iqb4sW8<9PT-;L5%uN58;R4^gJGWd;1rHPk#HS3I16arv$KR5nZ}g#w=U?E1g2z zHCT~!w)sz*lCBHGNtJNz7mn8tosbw_Z|9_=WBc^0CcIMUsCvL;Fc&8TNKJ;OKA9?% zmSC#B8wU$UKCq+n9Y=AEqLr#tQW*GZGptZm7|b?Xys#uu^4!tw)MnZQ9s0G)c;Re- z%zR;eidJg8+NNAy$SBcC$D$kbFHZ(X6FePONWl)KwW8q^ehGk`@oK-(Iq$8g7&YyU ztzgZ7Guj1lMvEbBpz&S^1}o7L{J!PTEX1>(Sj`osUNa>kB7%in#*N#`Y{VjZXGJ_w zZsCTdp1Up!-3<>ABg5Ssu4iW!M|9%c$iGy=&hh@;c|><*JFv1pvGeh8^<3^Oi-Qo4 z9ON@}3!QB~T>o@rxPi_$@-vu54)_$q&P1huEWZBnqeqW|gCC8C*Z1t)9u<%mIzK2|R@o9hipD48#f2(S6w}i^SIh3*k{ex=3{u|OC*Y)WMS}b>OW9)f< z$>I2r7lK>bYBy}s|Hvf%cP8kv{OTz#ubFyR2u}SiOQ7z{XuIoxFu(Q(zrx7^Q7)eq z(!cotZ3_O^Ds*Ee|Dm>#KPri-i;?{4yXA%8*0!LHDcFD!1=^*q5xexhWw z)^+ZU*}oYk?c#JB2J64N7wvEVwJ?3B#e3u*NRAhKB+XLT|v%dl*dM*^8Zd=;RSI=Xzd;W#syEDK4{;5VP+m0D> zK(9`y6zt#l1lT)|!ALCFJBOtEA%Y1;JK%#=TfSCQoJX-6WZ*VlZLuFee$<@E0=7Ao zFpZ4g9walLd%Qac(o>Yh%gE-^(TyqeK=dMSE=fKR!qDpfH1C05s{NUOAH8~I<(pl- z%&`4PD2z9jlH~=#(5&|0^H7i9Fo13)?H4S+CdqVSL9K{fEB#M!GDbL$vIZs|^Tmx9 zM%uIEcMYLErx#*SRvU(k!FRhH{m1D-bP74%ltMzqQ$~Isp4iJ7Q84C7G-gQI2;W@E z1MAv7d8cADqcqAHP?7;eOT3rJBdt07k zUn!+~@`d2vXEV|r;NgK2%KB(qi7sT51o7L$vga!P0_Yt>odc2*5xwE8x$d& zYzTLNO2~xY(I{7$5^^L1Mi*=QJR~H<(e3YXd{(~$BPT(y>o`~VsxAogdBt*HZ!hAQ z=;Nb?diNRrOzeq-f|IsS*FRA!Uf|Q18ZjMkkqVv1O(?7MenJ!T6>TnrT$M01$Wxmq z!OvAx1b65$smlR>*lYO}{Q+>Y;EZUd{SS zp>;>Lg+{~qXOXI^;yAL#=Vo@>}RX_TkH_oeY^-M?!K)c6hJvZ!`&_V|zq( z8WpzaeY$gLH~t(L;tzv=LpuxbBcsarSQq&fLD8pc3|38`&@Gw#CIxI&&`-MCctC5C}Z&@Jmrq5lCU2$Osl47^Rst{e8f|$V9?pLL{Q2aG^2No)P^g*A;@P=t7gzugMoitA?YE1R;hFa{ zGA2anj6K!GxL2^HrljUpjd~0xgvA1!zIx@#pKvN&>c_I5aDP+qh(KY+A3Ju;4Wzlu-F zmxyQjG#nOtkETM*O8duaVO}SWlr%^hzdr47=VG{c8g8*F_zV~?P_2-8P+FcFAN4$; zW~`k*Jq)uh3@2(gjZT&<+wyVfugMMHY@fazRyqiB8w)jyp_az^_P3R~!Qy^!&UJujLl&_#w9po14k#4pFiEHdCMZ> zdUs?I7Qm#h{7Lek(|H(T--CVXettDyVj$zDRk&xETtZn}Ml#lvofPHHNAwcP?|%_6 z5Wp&WL|uA3+j7)>BsnQJuLmAA^p!ql)vkE9#qz#N=iZZV#clLpegyP#EE&NsapnLx zoU`**bJEw_;pg-srMQX=3CtIa;9L+B(u;(wl{73E&oeKLXY2@hbMK>YDw$3BtGIs1-WgtC-g z`cT2S*WKl8>J1E*%K&=wcad!bSk zbyyHMC7N~1_B{cQx!jM%ox7!2zT;@bpO77|{{4)y^~41k@0A6~i9BN83K<_<;m@pW zVKK~Pl|dhjU%KSEtWOAtlHEfPoZNnjiy%ArgTfDL7_WFIX6pwUtHIj4%++z6yULx-c?3Ffs!)JLb zlKN2l0Exk0>JgH{VqWm1!Z*J@2v}@9SBMq|$^h(|b6rs&>y^H3vV8|{u*@}6oqm8V zJaOqWvz(v>LwkSv)l_~3+OUKcY%ZkpN(8uqtGD0>Qlz=$N#*;Ji6ldpZmj{5D`d2U zVFUrK{9M6*JI? z=L-peaA1hm$YxzoboaOYQT4kRP#m48Jv|uVI-b+o2fp;BOZ>-eV9U)u1MnrfKC~*$0|w2-L1+2iU&I5piA8YC~m0XG7XxHP)v>@X-Kgtj}V# zu{L`N7rGz@$C^{1CH1*>0`lU)-$Rb-cmU6hqxo)5sq6+!HWY&+$W0$6pSRE}+S=NJc0LbE8YgVi z^!}0|hxyhg~&m-khQZKgp&;^~W(jzNbhEOqXEboM}K_?0ECCy96aA zhytL@9~M<4%5(ZVi@dj|k`2U>Z9f>~i@Xvtj>5>D1aUA28{9uIaNgV7JAp_78N2pw z=xJwGB9R2thx+4#L$1_9)VWuM6@I977kuwhXJ3z!l6U#;4U9ZddZAL+-NdtfpmD zl&ps>CLI4(EVaX8;N1YIw;=!kIqzb?9Ocb4s1X4gq=?+e1{IDBYB{7^ajFbN$BiA_GsKKIMC;YPEBJQnNkspUEeT40lcSb?Q5h`JnZkbL*BT$?7E6|By6 zW^@qk^KX_tsyNSV_%dq^Y*vk=Tr?>CWMb1)Z<*-kukfo&#dhw4M8BGMzosXO(G>oO zl$H>4UYZ%kv|KmSx^4_cQPe0JPzr7aY-A!AQVZoz_QgR48N@6Wl(<&9*_ktE_QUT% zuje#Z5@VryUCyh3hJt2k0k`+J!2SxmPW_bHeidm>38=scyw~Rl*7v)sH`sSG8F}`U>K1!pT-F4-4yd&u|#q+=C}-~RH>kJr!LJ#nmN@m~#u zN)}8U$2-JSRaHg4$}qMj@NMUGyn-Xc2`q^P!nwBHQjFp$flu!do^rBSHRRfKO3rsV z5I;cuQG!|+?%Q)-E3rMkDQNR61#=Z^-Kzz)Q;1W*6u_@E-E9V{LYr%iy+8Q30SJYu z)rjezznb1ycH6vzO6!QiW*&}ce>h0%6$L+*aTE{=;qzM zsg%@RaH6m;OIS$AF7<5tK;*r~;7YJ_YFQF@S;WWl%``MmP<|GcYu7)7rIj zr{(UIM8YF!6NdxX74cDIbT)0-rrCMbwf*C5Ry=mDrM)IxB4c_M)L3eRO(*t{goFfA z3?p;cTi;&<+|`MegZM$L4`uDDt}e^NhYznp11rOy6NFNqG=d!f4Pk&Ie1*FNbu$kz zozVRgLM2bzKbh&UtLgzB{1GKv*dXw|TB~)_NAXRGX{#5JdrR0ER$-Mv5nC&pJM9}pxV@2TtYS)W<9m}q4WTsk)~$%N36C@ za3xSceGs0J@miYMcA5(zeg#A7tSM}cij)>&64c|EEH34P{~K7iLk&5RS;A0fe8FgS zo0hbV70%mXDh8>o6NUw+xL~ga-2L%tb7`^1JoG{O$N?B-6utA~+t6lRVGbuK`7?W9 zTHrDnMKS_epS71J`_*tR@IAU;8OYA8%;k0q6mKgF*96$TlytQDmS#sa8mrD}D05&G zEmuI*vO@pQr5cBF2IZ!LTF9;Dfzd)XGY~3#1b*S8xsf7P$}~sd!e?z|2&i61avB;M zUW3M5nGP%ckW21+e>Dz4nb3lhcE>+!3h!ULuLg-RW`Q8PJzV?43>>{fCuhTzxdM$Z z7Jb4w$r3x0aL}R9o``2`J1uj{OX-Bs$5Tu~*V#0D$QMG%MN>b8bGOXl`WekqSL>IVyr==k9(I4tRVg3|zhe#9I}n zzrfhVf(y{xVvJdXAxlmck@;#Q0Jb3$`)zpMrDkyyE0CNy#hJ1(?`8c|jT<+--&8UJc_9}(@_H*&wS-&7B3b}4VE4=7#C8NUE?%iquZM+tik28i#c6nXc)@hAj&GI_UvV_8mpG)N!QgI5^waMl-Lu zD-%FD#my+-8!yWnjep)h4gkN_A~CWIO3 z9X~KzZM-IuQ*5PQb zds~(&xZK>DjV?6mGJ+L?>#!k{7-r<2e9b)D~-T`hDt>2h}BeAMh1^%`hw! zkNZTyL(T-D9P|XlRfwGECHa(1g7Obo0_^`>Dk>7pteA5KK`$X8fwl4nw53tase3fD zv1e9C)-dN97y@>1p;)^;0fREbnV#Gz&zD8;Ue@xMzyDTKhKm`H2%wj1jYmN!q@vqi z5lLV+QpTAKy0^-yARY_T-q;FMWNF^%9L*x)T9+$?qN%>y4pSqyJ0!i@j*}qhdmdGV zGj!9VzPi-Lv_`8D79DQW#`~(ErbuI-@OOT@(E_7EV-S(*c*RezZ0J~j9&Hj-ZLCCh z=gVJPvp~zCCr;3Rjl?8a->z7UF+Q2)j@R1apjB}cySVi?*Aj4Dk3q-;ghLm!T8`lw z4-fCNPtM~Vnk*JzHCQ{}$HW`}(;}LCDHsm!;#6{njc+>KkdOu6M1qT(8w07bTx8VR z&1*9}H#_j^#Pw?yZaMj0UY8=ptGz~FLLqiigytxId(0784-PK)SJF&mUy{r}404zJ z%tFi1qMlxDph-bE=h5Pb#{~z zy*kH2gma;?ac5XDBRJE}C%jezj~UEqdzxP9mUDmwrF%hZtJxu$?NBN8Q09du06T!- z&D%m#Lc+uKO(pqE9De7Mb^HYAF>mVYF4L)>U$PvB8v;&_!qbd2ZJnAb6TN?X7t;y9 z9s7T zrpmOq9`o@{j!{yLSNQ;9nIO;06c-R+dJd#tG)MQpDc;{Cr^mZOARKbVFgP{jUWw-U0AHUWt^`Xbs#?7YAZPgfP87dWQBu zm+5|}8&eBIfMqw!$pg0e1FxF$%i~o-Ojeg>>ytH-P;3Wx8V5T{q(`(H{h@Sq4tgoj zW^c(bj+WvQ0#l6(WDmp0s8fn;p>kMN%yyiVm&?3x=r~j}mrM`*wNMqdtIui+G=1(t z>*SQVQHlz0GEI+zzXQ-(I!^7Tjet>+q#o8ba%UeC^fth@7C9@+m}A+@M-(4KD)Gzx z?MaEj>=Nl!g7-icy>o8l+6x9%S+4n(mJUJ<<>UUOjLGbI}2;;$% z6xn=MKP4xTt$KE-p4WRe)i@yvA1)!}J~wuKu(Kos+^98Duus$J&%Y_G^1=Kf9ap8k zqJIsripN-1r8uWD6z_AJJuot>c^j%{1VK9FHzw2BrD~^{K*&9rNI>56?Ppj|Rxq5G zi^cE${0tVhidj+1DL|kV6eo(b;Ic%$7PUA%_)|u_9{<6veOm{-3hxS=YoXxgXg-rP zA(IxS%IGs3@>)rVXvlZ`=0>nn210=PjC+FGy2dkQ9%z<)00W5wpqU!sgF-G)i`sm8 zH#-BS+o_d0eMaD5`$5PGwfhoL2#ZVBUGsD<#2s1j2>&E+={>WZQ`5o#vJfMh#58OV z8-!g_L=K_NgMQj?Khj+|4=@K{{Z8IZr#6HBpbo&(`u#XWCL~258qkCxf_A|F{xt}~ zU#4y#iX?J0MP}9$Py~s{O_7{k0dFA1(K-#t%(s;Ten75Ga_dHv2dT-9TpMxd-&JX$FM|aBmo-%11i`j;bV3F6u|*+?(*52 zT3!CfNT`l6=#6vIE*ELZ%)hxH(Dgr7{r{UoLf1s2UN4e(7I^1}`w1?e>EX6Xe`CL_ zFp3>*QkS9~YN5f)m_SD5FYuKEv`IaKG67E$z^j0|`Y0&ArmOq;sual14x{TnyYoLO zC;ac{zt(f|-;|U&ymeQ8u1w;fBpLD%badu-QDZ8q)B=u>PyeJo(J$b2MSi`1i}sc- zX6^rUE&sKe@Hwp_&)0eeYP`zhN_}KHu$}THE)#wz(d@fz79oEIhSI#@f`6UAqEdS; z8VCkA(wZ5Z19e)OyJ74tt;CQvQiatGPGj`m_r%{_3h-|1LZaNvg%d{8dV%DXxPI=d z2HA4X1TQQKmIaczG6~G4D8TBT16jBiP}9Y-dY@Yf$_K0>Aa&XU%PQnef|KL0-m6Q{ z4jzsAbQ@BUSNp7ay*R}A6l$Fl#xxW=v0T+9HpKzc~fJtHXc;rNJ`2sO#`q zdkC{|bM|AzUwg*mt~cV37Q;D;SJIY>VrnWd)C)eQY8LBPA~{vvn`0aL2mGKI-3sy(yh@BmdE_`jn{$AV>#|uR(M1N>x(*9uDLZ@&IX{P;-=Ej4V z^_#BIZpuD0hH*Uw+Papw&1*a_UTr&_uhg>TWK`#Es($WNqFt^2sxp^R(qd<}1?M=l z)(n5#L>E)5&LMfDFaYiG8=bWI6CdeZ9+Ijdko^j_!of5#S#8)Fx%$LQqmYS%|Dty) zVW?I1&GZb`nz(8FYoq`BWd2n$!-5URk9_(Te zNJoTYK3t($ef5{hA0k5Vv5orOku!uImQ_5 zBsYZ+w{h$PHz3W~9fo_<5W@aV;arcobF~`LvzWV5)!UWJsDH@=E2yOo%X6KB+-#&0 z23hjiUw&+7KPyv*Mt4Y+x;!ISC*$0;7ngb;=NF4WB6nc@!L;1FX`A6YO?KQJyL?Sk zcB^PIb8MsYw>{cKcR;@% zC@_S*f>dSY6Y=%588ZKkHi{_PC}cJ85I($&NHc7#rT$HvtxhGPQ!{;dR8Q53uqe$U^q9Vu!GAGA-QH3rV`mJK1s}b&x-v?13Z&7H* zK4e&{o&@8#xHg-OdMMF{#`YJujIs>|NXiyCN*$si7F2Eoh^r-~nY;WCiSD=c0(qz_ zrlT%|_}O{6vd3dGp^y_q``GbzmWt6EM4`w-bML*E?GSX6~t7@?&Afvjri)W z<1b%@R=_z=+^FkqR?khh&~7p=?sm0j^j5fpUd|Fw9l1IUvy;BAY())L$rflVwQ>%w z4ToXgn%EI|VoLx571FHT+Pu##7Z4QMAZ>{+uQphi@YF| zZiP8szdre@^GHI$HWwiC5tN}AbaK+g+zWka9UwhXBJac{?)dFz-p`fiZ}-AVT|Wc6 z!&eDSO@v$RkgksA{_aAjz|_jME7()u0=$Qc-i>xqR6!-`?NItmaZ77RQAPr8AigX?y7+T0w1t z0Fauz5-?_KguuaMe%wr8zk%9v;+8Om!8k_faJ> zusOArT3e_EKzogWA?s!tfcY0%um;Z&$k(j>4?g5l4KtuQ3boJ=$>PXEs?vg=(Cfcbw?zBo|GEu>cA5WsLuAouum3xxRA@(= zjQ_H_ZrxWwpKN>na>YU81En%CzPjz57{N5`Tc=GJly}q7ebf8fAJfnDtSL3^2~QBV zx@xf>5f~YFV0#Yl>6SnGXl}tE+oE?{9(-f=?`p>ww#z~Sfum;7+{Jp`YVqx*EYC|8 zKXi53D!E?lBgos%o0HrkAHv$@QjHfHiNe;3Xxpd46rH`*twT&j#WKt8*H3sC9`as{ zJW{X&RI@cNAq9O9UJWl-^(7^bpNB6r`e^M<^{1e=Ur?{7sJ=V=I%p)MRsbLI<@~3Q{O}w$hB7`MnUZod z80mU}6kLanCw8In66r&FP9QFpkrMR6$5xg0?Q2jCi$3t|1Ij44U56;fE(*Ry^P8-2yJadwxPZ`m(dhWf_v9419$Ip`xRP zsyPttXGA6EjuTy{o!f82@&M5CFP`fug+`>EQF?lO2>50)Zg{pYP~29Db5D@p!cbgr z-*8fN5R`b(+)h~XgEWv?_kJ8wbR-BFRYHwa7@EM(&U0gXk{8K7|9KXoPg1U(qg+dCnCot@@%VD`p$5eW%=MQZx2vj}OIbeK~yfHcG?a0}q_r zQSa4JV`bZPNrVI)9pb4N=Fyy^lcU zAC#bbj6G+Dxzqtue#p=y?qqLRh)J!e1GGw(HJ8%&3C+Pbs>hSL zk8d4>Qdx?Ia-HObI4*olptcb#bOC3};XQ2>|9YTZq?SNeZ=<{L z1Cazy6veiXI#L{m7Qt?G5G%Nae(#^RV88y8xc5$d`I-uP}MWx zdbc5UH{BH#KptIlsfDI+jhweW8<{A;Z*bV~dBmO%kB`3cMvu1?4ynzE>l+nD2TQ8hK|BPOj9*8^F3Xe+VegvT97;AH@ z7WgAQvurv8TG8K`ii-{beorvc2ZT3&(fcGq5ycM;0ctCnA5j~J^gtE}>>i9HhrAN>qv%am>Uati6_#J}7rN_etUB+oy`i~*OBGk54c=w_KU|~t` z&%!DhJ1w@&jIsmY#5XAL0@AK9``Z50G1=Zm?sXG2^Nm+1*bJocD@UIj_E#!=;erKcX_ic8WU0G z4fMtWB(F?SeJ_;2N`phlmIDza$TvY3fYCtr{wn;FTA9`C*UYeXR-!KxuPFT~Obyh?k`qEE{VPq#mbXdiSSMamT(HdHcM zGNxEiLx*Ecw8!2I&5oq*`mp#Y7bvOs#x&0A7uDFpHqBeaMdJ>KtsdevkOIC+r;!pZ zXb6lg5y ztemz#goXf3-&3na!;ivk1C6^J4bupPfUDPL@jlJfZZ{eRL!7k0cVKwlw)G)&toOn4 zmDkG#`@t6do_ApyIv;USHE$2kQK^n^cu*Bp+m0U*v${%y+3c08@hxAf&^<2Xz0Z@} zMlW^HowGR(Ck!(qJn-dl-?&SUKcajs?pfdAQZ6!*>Rg^CY2LStL4jO+vlM;|h^=-W zSB4_G7BP52F0(xp?X=D5Aaj0U#%^Lr7a?*dq{I+8;>@_0Bgfs(-oPFJ6k#iZMqcAU z)|NN49GA=IsRxsLM}AGSXsLspb+iZeJA~H+G5rkPDdQbobtyXwky?9gKVKfO_zHMM zkye7R`#^t&a+vz#a$x!MM@<&9E;C<6Kk?Ue<`cgNJARWu?VuMI2I8Bh1xx$C3Ov$7 zB`beO5Z=146+b5=Hzj1i2Y$R3D{LTWq2iD_LC!dEPF>nE9UA23^YInHB(hIHKetKP zb^9|Y1VPD!5!F<=7 zYk%p^Ouq{Af3qW-7TFRGsi*Yhx^Q`l=3wY|oXBshY@d78tB}C{($N z%v=r}X-d!l1cA-yAs_4Lg=>%Jo`42`FPZSHiGjg2*!F(jm2)bG zZgc_@1!g&oD0=oQw#!+0w^VPQCI0d#x$q*|GQh2cT83A)Cb!aq&erPL8Xr52PwI1d z2QDEs1_3I%=-k6fP|CjJ)E2!PNAFU$&?j)w54B=zvC6eI-;w`FeEV-pds4mmk5DJP@3?p)GY+p-7`qV177WRF(eHh(-w)kzRs)rmJNH+FN1 zND7wR(4@u<3Q8rWr5UbbeeYfxHCppVG;(OhLgfn%J%6TH@oWm&l;#n75xK|UdBuPI zv)J8s&wXQ%3%;;0IXguvODZt%h<-h|WjjY<#g!SLXM3xs4s!@h3dJIcly%PHRE|Oj zzaaACF44s=lEA@Y%U;ZgHmQJK_Mdxt?u`M6fo^7p?x;;E&?3Drxe`%%5VE{RiUL02 zIgC=tPuK0#?cXFDL!*i4IxIe`&0;QT+AV)qAK6PJ@3KrLhmy5>q=(M6IkMf?N06Q`0fRIO5v@R?FJ*f`s$MkWR zH`hOL6Z)1ki$15vzO6OXfh?76-f)X0C9z7r($8;LV=|3AFO7`XAT^HmYydf;q_~+k zWu?tGCUgGgLLM-zLM;WSY?c-ms(~-|&0rMbQAUl>kM_tHc5r8BfGW}%+1m+}N#=!i z;kx;jPAU8DmjB{zC3p`OQ^{`xl1wBey-xeAb`psB@+E(wM}iV?t!1K@4c1|)KzRkL z-^C2o4cilCIgm>5t2y`_t6ju7uU;Vb{Mi)t^6DL>sU;vkG&EfpNZhf{>Qme9bLvvU zvt0C-qVp?!r_ME+-3hov5M-1uo0+ij5Vv~8P=sPFq^cLKBaXYKA`L^1Tr$CSJNt8B zYkGRekR=7Z3YJtM4l;tTc3xIe^c)OmWE+b}1X_=s&CUVH2+vQoX{Nf>`OK|yCikec)rPgDY zHF8>JB2zqj;!AX3k?8`%QItU86+h)>z5xgl+gR~L-tt-#5k>q4qMoqIzvk{H`>~MZ z%S8VA&85rBC*j5<#PfjRI(6iO7%0a+Xpz^DPr*ZI#?!9fMP&1K(EhC5A%zePlpriL zHU&c&u?!Eg#CiQ6qcuk~yW&;4ZJYJ8$|k_lm-yhapZ5NI z>Z7w7G#*lKW59MIgn48gZiZ%$>gf1 zgy%dWLQhj|mvvSIMNM1AR0$ervticsNOpbDbZF)&ghUlZm6~J#4Tz4UvZTlr45BW{ zxW4USYm!bq(qL!1&LhbEo=Udh>z)N#g7sa`(}T74ivn*eBlp<_p5cQXNaJn zb(T$`L3U5@?5MnolV2)}r0Z&+gy`8_j6VIreh`>`!k%6%vTe6i!nH`J>+>_*B>qQq zRh;raUn3DrX}qfyS)Chp<|y!RCGTg>$iVa<{fg-D8mH$W>!TEw3x+$VL-4Cm>9pkt z2Bb=4jcIMNmypP%gUTLOSfkdvC^cD3tC|RhnB>~)$Dpuv^H=FiJir}r^D-Oxj<>(Y zJa2=*W%@M7Nr?E{mXMN!`}GMbTW|Ak^Ju(}gWk~8=^`-`hofDjR%1@n^gJmZzQ=*E zneIe{TxO9^uOfGVo{g$+m*nY#fOD%wcjFeEmSLSvi6BH?eP(2B?DbJ$Su|dEex!i#Ye4J?tfyJm(*i*RWtym~%ewiP%jZqY30y)bl6%6d7`KRhJcCc? z17y{rEg(^>)(J{coBdC3^Q9k6MA1i)Z-oV9UG1U6*J`CumEP+RD_W0#xD=2q0+ky= zDk|_D$X#0gIKYbc;HXdQaC305Hvw+W#9ABVqwpSB0gD5VLof9XS18T_$;BojV%G0{ z%$?98y!tiO@9iYlbn$cEYiEcB?tmev2GMHFC_FqQ1T+7X;<11YjP0~pa!z%XGj5!l z1aSj){>xhnq5%Ysd@_I-e?Sa|Vp^&)1zCvQ(zE@1lt<+0 zw|NI_w@+0`Q-XAQ7G0?Re`W|3yMD8WJVUF++OViMb4JM=j%lzEkfX@gw=`ldRYEV)qtLtKpd#OvQlpcYXdIby>NO zk8WI09Plk4X|~!+|GVAyS91495N2vJD*}uF`3>Nf(g$$L8`A~W&@!wwzwwJ$iE#x! z)pgY!LJ@5{Y@m5D@|Q6PNw)gVjj@w`@B9P4}P6Qu6gN1Jk#K0W2#w7&&=c>fmbP}Scu!- zoJ1yJb}3bjvMied!ErI8t`>mFeE3iN-hu>9FO?h@6jGvlPYt>&yc&fZk*0D_U!TC^ zhsFT41NM|OFZO(!R4n)LiC3rpo@>$Q?oXVzZaFb4Df?|1;^7*>z-DfTIW4wTxZyst zeu~9pv0k(7eeHv?b)>qPHzWY0RuMjn`gY2u*EHx*LP)>3>uGN&(izv7duIe)AeUA1 z`%F4C(Bg`~;!v0fqQ&9tbeRq*uC`#3wTY*RFwd4wYn`cP!2!N^-7n1fu9oIikRZqr z<{sf)g7{ZVOO9O1^}MIyZ}~NL*oD6R+NDnWgIez2<_C0Mm1_~bwTUak1V%Q0rW#7& zAC2;q4o@MpSCcD2ER>Lst4c;Wvb2;@-z{_1l8=o5a2Zp6O(7k)o6GSm5^o@*Sj}a* zfP7)e$R5&f&9rg9aiDmgK9GzzWB}3^7Z5N54j;8D!W}SrSQltim zg`ZzF{fe!4^vwI9NqO$pc=l@>By=4V7YE!Tf*aRdfRzW~SQ0qs(gYOcN}<-*J{#b3 zzQRw>qd6Em8N77aeQ1B8hYi9_Ci2uDI92!T$%&`1M}_6cANQe$WBCqf6Gr?galxkuwpxG zk=JswJ$y;4z2B1%{f_vBNOt*U>(p~8!`*iL+X6;;;9ndzUMwqFI+1O;pnmon= zXttR#ECRk*v)^_@mZ#uL4?h^JM?m_A0IlDT($e-|`Ovp^pe(g_ifN9mNz5l-|BIgP zGZV@?gUhIKuP!OtOAf+0@g0kxz%<#L(vo-CxnFRxn%H^Oo9j(Ugr9(M9rJUyi4G8Y zzQN>6mz;)I!qb5m@LIP`1}MXn`xsMkVtwT`q<|03=E>oO8cVC2P+AIuAPCCQT_r8K z5;J`KON;W3IoWb%onG)nC#*oau-R81QDwC;P`e&(<%3g?qF^K2S11x|%qVix>QN^m zdngmp8AkeBS;4MO;Igy1@y@{g@(7x@+! zqq7!Cxm}F0CMD!jL^AC%Is&;E7 z`PjX$ng0voTNh;~x=5wW#}a+QlE*88azcSUn*+eCBp@hZ*SgP=4*f$-+2BD{Y#j`C z6gK2_yc3PDa9J9I)WDNrpVgM`Uf9~cOIbAPwz{bj@GO%n5uN=Tv*?WZ`S&b)+g{8} zWDOhQ(FWm5&;Mjs@h8T8ino}ESqfgzdIR*+`o$B&oX;pr*iyS$dp12p7!06a#5Q9k zPFUW2>cAO@JRpV+zcvmdNZD5pnRFqAyhWOEbZIIs=ew-}-Hpf8GgHV=4h2c)(jG#b zMI%0m3#`F+ra_-x{4QAYnbB}zfz`9Qpl39Q_NvuY*AD*l`yp*U*8bsohrQ*V_a#RD z4s#=pG7B+H#4zLb>4Kvh2gP!H|H#%ks;WY*uQNO{FG}f6n~GED1bJ z)hnH;Q!v!>j23V*EgZ;IA@xTL=CXLF+i7rl9y2XO8TsZK?7Kf8Qqn|3r4B>~S<|-U z<{&kCeAlU)CV`Vz5p?eoKfZQ~QUO*t&#+||op}V*x%3rSrC*SZCjaOHCJz;-k++3_ znlYW@E(v3q~(V=li$7dC;Xs>f$r8(l(z7VEJuDX zj{X}eg+N4O0*?-%l)YJS0re`l(MxY{^~~kLNiZUt3t+O|R}F+h^~8XVQMkJtFo3Ck z=}{9}njE*cgE)#VU=F^p9ON~~|4a(@_k*Gcvk?-Uu#*RM34EjZ;I&yd=3b&;yb({a z+tfQt>?qLG;vlTv2Mb&Y=Nj2g9T-IpE22wdPl~vD{5Si>1ZMFekEG=e(yB&qwD;U% zfbYenLCyMsuoWyKuWe$nE`=+Xqu~4xMjf-2z)f`0N$89S@PiYhE(0HOEtgX-lTZU6 zM6z`Pm#;^-^uC4)hnMcM#ikHO*wxt}^tEtS$EDeIk1iLV4aDP*(#r{(E}_X8AwlRr zRmM&rLtr&F8PUSHxlv?=2v85kI@qG4<51!=`7)W6IjFfAMWTQI+H1}!wMFvEE$Z$5EVFD%Od ziePgrMC&UZMXS}ku?$49dlCPyKQ7?X>N6P>2ZDd7lmR75ns>~j*gT_Msm5*d7*wv3 zZ)Vb;)cpwCmB73@ue^ZRr$<|T_M(Fnmc2;3AX+!!FKaP8boi1fCzn6r^!*1dSl$gqr}$V_0?*7Dfqu5M)R3ss!967;WgL zhXn<_iTZKfo0gMoXID@#5s&3VH$`#fYp>rO41G*3g2?&<3XAlJ;>YNFUiye5kvH-n z)@;NwuNFZLuwW(-l|v<$rXo~94wUb<9o=>nu%W<_1X8Y0i=%4Z3USn>YD`wa0f1)vFxTmGb9Rq z6aOj!|JD!?ei}M)jQDVImqCatm3JuPiA@MLu{~DB?8||31pmD~UdOJ3SlAu3rSkPlH{I3<96B zbJn6-MCga5cO|dgs`tWFx|BrA_?G(AH;B`J(dXxuPN}*0Da~t8AAA1XIaB+HpjwBe zCKcf{caw16yQ2wjJxmN1hy47+Zw91)Wm01j96EW+>x2j)$JuMXvqx$)@%0sqmiFe~ z7>OCy^U~I`qef;aM-90dCWY>IcN3bc2K=`kqVx6hyTHhZyFcHv58ot1ust6THB&lZ zs$9`ZnZT(p=l=EAif-YrcMBP!cK%z29kH@M(>~cg%`#PTUTCkMk$JPpnDdWpHXS)B zklV<9$sDKI@meqQ_GfvMj|u*hlvS_q9;-`U)9=%5DBfPC10O5C)0M+@?78cn4{IEC zVy%;9gYR6`*CC8g3iYjeJe=6}i?bc?((2G7YNwtwFEYP6<$>vxC=Gw`K)NvaiQhtH z#eB}NVQ`Vpa7vKhybsF2(A&vu+l6WWP`=uJO5dOLZR)3a?dFoA&$l z%Dx?+d(m5FGXC~)-}ZUIu~QuNhIKr#hpkCJi|5NbW-phQrE(2B9?85efxp1@%f-0_ z-JG=WD5)=0Xa3NjzTP#CI3Z4kbJdT^KW~b*!XBE=`Y|S*?&-Lj-Ti9#^%1FpDD28O z#l_%``e^cxG1sub{F%ImFM=x;_=W74gIE@NK2#bGM}{#srX0n+7c$;U{v|ZzczqSbUk*2F18IjzBUxW+Oh6Q5D$ZmjRJc!#dEUI@;TnXt;!y^%P`swe*cwD;XnO=jKt8J*EjQ5h@)NF5Y#=ta5|M-UMOQHmmEq$6FL zbQ~2BX^J#y3Q{b9^b#P7K%}ckFF`^PLPD=0w4c_;p59T(2eK_TG?EqJEUpbbdh7vHuQcxLcIU3c{GDb zJZ#v!nZ)*XS>dqp|+v+mpaJz`^lU$e_QT9}6` zi@%|84p0AVuWD{^g+;C{@zXf%camRBJH=X2NoTgY`sUoQ->95;^+D}~yst~MPjrP2 zT@oK))lAokjH5khqp#f<_#Yo$VqBIGT%S&%%tZdSGDf+_c-n zb!@m@Wtqm^yZD3C0;4&qH2WC>PIbWAHDbr!!w1L5#|1b2mh1%^!wQK|Mo7wz>=XM* zN5OBkbQ3o_q`Cz~s zvYRsMJ2}qJOa@D)f~9aL5Pvy1rg`$@mYag1CiADO9j!7OpIGqYgO@b!ZhI^EetrYO zyw3SNDgk6Y?jB2?0&a=Uob=o+Iq5SJ8P>sr^K6a=7h-sz`+26XvkQKjuTD_tc6hk% z*Bs}t8@kf}CL{O!{>oHHamD7*$qi%6^M`qnZtWMNe}ZbpM3cF@KWMpN{Trrl+KoS!P3M6& zr?S$HQp`(^`?!2g5O4D8L+9nRwL#Mecyos4Bd=y$A1xW1Om<1)0-whZ-P1p4L*4ny zV1P-y5fq3!A{Tn1disHUjE<(d_L;SECG*IPjJmL}-mQi9vEr~W@HPjbb^63O;MBoH z#`Cl87rJsw$9VCUCEygunUr4B+Qb5p{?E>$ ze_fu_b@N+k6Z#AW^1i=Ky5aoW`q&E3yvvxr9fx@@BV4|zVCcN=UWRJtMh;r<01olb z*ZY52XBn!)PBF}jx8-yM7tfK#sA${kX&qire&zbR@}))pPjm!QT}S_ZA5*Dh^!A8; zxmVjM78X7I%ZcQR@b{g=3Klp2`IW?zP!JRR%Ko|{8`zX(zc-A(kDomx269|YmS%h8 zHdaX-KmD}Mv*F#--GzTTF=06j6-Q}G-CI)^+}+!k%-};UI`x_ZoWO6DtePFrxfj-} zqa!SmqP0z@5ra9xr=@;U|8Og#bwi!e0B^BPiyRj=#1ASITSDySCVS^rDtOKMh$7z& zeT(mxVfJKu;RGk$8yWckrrJNxNNa2M6QP#GJKLSers+I8xArvm_3O^L`DU5u({0&p zk_cYBVhhrryr67p^CU3SdKGk3U%nZLcCh-n$$b<(sYI5Xv8 zxuFkiBss?jxE;;)UOma#_5r0qvCh?0_x3(zN>h||dPH{iZMO~foOFGfd3Jhqn9e&q z+?1Y-n=!hIT&;@qj(uw;(H&p1oW$|f>r55^PUaJcVl8LiBt@r9Kghv9dSGjXjo4=@ zOIIyotX3+R$fyD42|riiy&m3G=tPYgqZPL+77qj@Ki=2am=kn}xxn}6zGt0KPbL@_ zzPY(6JFvM_n2ujK^m1Tk0N!eL$PdTWQVJCJH07Jp)dz8Q_Sn@Hd{nrSGd2q*ZO?A$ zeRk}J9cf#m;Kb70SEx4~2Wr7~YLCk*(@9XBt>Ss{+{?7yFO}d@l$^n}KrypPnLGk- z^}X|=cnYsbDnyJ!^v&YS*uYZ@!vG=NdQ!{1*(T0|pEvcU=eEhTn}q%rTM+%C`Q<=$ z5XDvnmI`e#dKHB_7aPPID)c2OyGfMQC&pVPne zTDj1@hjMv!Emp7C-hII=+r^>gyv>M-gXyySH%~WFGLQqIl<=5VT4A;JT6T7Js9g1h zfDX)hY{T{cOrTS#7St9cqM9OCXU@6sapv-+VHu7dPetB5-^NyS;iuN8M&J6)-h+`7 z?X$=VWn^sef!P!5mzRt2{hxcgp44^Yl@=5^`=lpfeI;-iU?Iebt2W+#1s{xYf$0%p ztJDe0UF^s;`6n`lqx@4vrqz|Pll~BCrQSK6wq+j0Df*ngu?g48$jy})&JB`Ab_P8> zKXKKiJlT!xKv?tI|4dZ$x2VPNj>zvJ0QJ*=`7pb1f;dHDZeuoR|V-w)p% z`!k+)KfY18fMV~?gtlhuf%zM84Zh9yCLw^ zFQDs=Sit~zmCKbI53JspL$hCeUWd91x~RUJ)yz>Ycy>!eRcps5>-2`PW4{wtb2>c1=WlN1jG_q6A0GvWASU~>#$`<@NM%gPA|u~X9;fsd^lgVtOtdhB`B+=z2L zGG49k_faP_yH&VZ9<(+mBYzlsNW9i(a&`1+y2!Z|7v&abl35`jHxbhn*H@;vU*iid z582n;j~BTQEw?uMx=6lYP5L85ek+Z8aGv37YHNLyfvMHCH!ojF=$R{X36z`aa7p#N zbS}3-2ymK4)lU(5k|Id*|K=z_8_&whBd| z~V&wDCkG_GSI#7!$u79AlW9P0hay}y!({|5^6dPjrP1SnevHh*eNs$hV zxJ`k+twDEsMw=~Qh4b5f2-v8`JzK0{E~)mmO61MpmvHT z_nR~BS1n*S_Ci4HR;SBe+WLP6}htY^=$S_mN0lNz^?!pcs;c4+`HzXUb07?FnKR$UFXcywVC}$uQ-Ts)`teV zQpz*mgQTAF)P`5-D&Rov5N7n_P&}a=(vBrz868Lf)&Gzx~m zmo?fNJ&|Q%Q#D9K-gGt;a$xMCc>M4#lLcrGng-)tLm_ABVB9Pp_xIP3lL?V9;+#~( zVH`j#vbr(>e`=`#rA+WkMQIp?iQbds z`LT|k`laWw;1K6d8SUP^fB?@;i=h{2RPOs}(e z-WVj*m?irU_swgX!_=mX^Sza&=h6p^Nwtg1{Z+E*SsaIB)36qshfLEtlU=q{d;;40 z((P8)D;7Q$dUiJUR@qyA(zM-ykF51j52g8&uEbnrHk+AwY-nJ{g`jx6h)K*KR>Zte z5tcZQHl-j%V3AU-d|fvGKI)!t;nQ-6s0$1Oqb8fgK$!>l;)ED%T%H!u8SP8BFUHA< zh_{$6yF(t>IC>ODECf;mEKwccn489WV#U`wrvAIq#;Rk4wWR_J5|cQ$XC%-RPqq!V z<#BBb2(L~ya`)|a@!Q6$)z)7By&T_hM?>oAFlCdPzV)bj>ums&(qQ(}FSKB1P3daS zF-smLVHMhQUH%qLo=ErhEp-$5H74cVN4I1AKbyj@Tp?Gnm*9!gw?5~L!oe@`)WPGH zXZOd{tjEV6;yw*cdOBeJ;l7;{H4E2U{wMGK%!ooqkH4zG@?jxZ%StwaZ?i1J;JCH>s)Kz8E7K zk*saG;(mWOQ*4Y2Da*&@v80R`w_&%_<+Z2*=X>s|i&F?vi~?W+A0+tn9>nYn-%%r_ zJp!DwbJuOkMj=YwT&Xli0|_R-`E@un$+9vFlNJ!n*EK^#oIAipV2NdZ%kjFYy`AHX z6mJT^#bplJva4d$@CC^sZa?rgg+N4Qd%U16jtPm(c$s5@qaS9})g9$U4E1uYg(n&V zojE$;x3;VFSTp-Uhd*;klM6c8nvCN`<{qR#DRMqlgs2@q6YEiBBk)#;A^;N3{I0GX z5X!&4N=s5%zxfzIErPk-`GlW~x4cd-q;kk2ObQwdfh;MteWi+NzikQkT!O-CzQ-@T z?ed=oIkBtNzpSdfu4pR+05n||idY8%5lsPS)U^qNowowPhB1QS49Je}f-xcqK~k2v zSNa-iwbA}v=2Na$h$tGvcz44)l^)%!4>c+|tk{5_zbX8DR<*w$$Gwj#8sYGHmiA?oFg2KePLwpEM2! zP-CBvQCM7In+%))4x^k_DyK@Pk&>q5{Hg*RxRzflbB*-s5&VyD=S2Qp=i4l-C@cta zC_q=}o~Y@)nA@ZGYWkh!FVMO3M&s#iuZI~`ft1T=||u)253yX z7h=a_%!d|@v43c4)=G@H1LCHt^0X<)$Z$o7UNsNX(2nQwtU%m&z8KFP4dMCdtmiWI zC(L_|=NWa+LIdWr$b}w#;Rv}>Sw$n!&p%=0ib3A(TCTehwnHb5oQ`4}o>HsS*Sq#z zV8A1J8DP6KP`yICrlMy&Eqer%KxEOob}d0zi!%=zL=TikhaQ(O+w_6`2nd<0EIL9g zK28#{N}gZuNK3EQkvr~%)g^Th{7K|Y$|T@$%kjsW(Tan9z<7~tVn~}Ql)W~QABaPy z+;_v6p+fMMF9X9zw)r@cWC6mAp7F&Lk<1g*3*o($4&I^E(tYd&0PPV<l8K3 zE{zbr9gGkb(I0vcjt21Vg-?R!4*@g`yU45hiSYIB*qugS?$&Y8l{?PesgMI~WY!;i z3BCmv0D;uz2x+RQ$x<^KK5Ac&Wi>43Suuo!ES|4MVdG8d&2fJ~(g zJ4o_8qZ@3$r}lM8+nnyIx7U)d01XLQ4`Q;7anZl* z+kw$hK%p80?}ur*FN*m1_>gljIFO^U9FdetN#4covegxGS%RLAATpo`h^h<^dAkCnJa7Fh6mK zg{8;n{MV0L+g0d&W6L0jdZo_-$426fIkmAoHn0pF7t2S1F~4Wfgyl;3)*5vi92g8r zF`&CQk@CNsqde1YinaVaen`?~>rZ4^D9)rxE1@6@a@#RZq#>txFsSI-g@Xsj@F;xW z=o>}I;!!jg+y=u{1b{DE1;SRAuX6Y>M<(2|zvn_k?qhjT!~DQ!L$#rglQSuia%htH z-X8v0+iiFNT3jKIdh&z#Y8Q+Spq?pb>RM43*i(3o*g5~c9}mYikCFVU>DHFU+v<=A zSxQQ`JJ5~p68suM$UoiQ0n^(-e|Z>F{~}$vTWpUT7$aq^eKpg6vlND6`q5dJfG-VU z@@1|r0&H%iSIHSR#46-MmG`M_$Bb3Mh_3*II{VWZm$SaT6i#-j_NBI@S|{>`pmAMe zr>z&;{&s)F)OXaf?%@VDyReQNU=P8@cN*XWN@Q?$R@(S})KCo!G6=sP8n9F&R^yxb&q@QRz`}K!9mpqA2`Lricssk9b&;B+n0eG4fXb|V#z0Bs0uyb7 zm_AaB{H|w6mKNrmOA3)kL_*N9lED20Qr`7d!t5mw_qYR$WR^F-;e-LJ6_9T=E z7LNuxtn@@pJ*5Lg{Tb}%#~Xc#+R@X8QKABM0&+WucLFMBDWLe|wMEnQY4>bsVW&u#J(K?(Yq8JtlEWt&r0|@;m5H zBYNosu*dpXRr5|888it$I5vq9&aYg$5EIlY?OYm|OOR3;IFCXEQc@6Z!;*0_R0pSR zyTN@}j4w2}^?+*cG~lfc0Hig|-6%N#Jwd`A9vvf{D9HX!1t>j zIyhF|x4M%+*Fggn4JOnY5{{=qBS?Ph#v*#FKkSTe^O02&blmOLcGZcu=5dFJ1Dpkb z=rv1&;4qOZP-@GN^Z>_SPX?92OPZ##6nl4`hQo=2D|hY@*DSg7CAUA>yti**S?A6L_0 zvhS2`fDm|QBBIytTU=wR-Ttje2P`9Hk2BVMxm@4gZ zxzMM*k@M7Yx+p*ymdMDU@Rr7inIa=jNW(&;Gle{lit$B_yTF}Kytv{fUHo;$Dji^C=Cq(+k&|}~xiIC@L?XUw10{si z#1ust&IW#Gyzqb9XLao7?-QpPpZ$l~gYVzt4gGtX;J->&{B1kW8-H8A>|ZXZbJlXl zgf$jsO$SFGW!y&arOO3oHK29#1QQrLv0*-Z+Xogt5Q=N|+U1Y&@v*tI9V4H%MOa)i z5=<7SB@BN{!yM6K%*7r#Xb4LS-o1O*ZEfz0V?v#tqW_vJtc-61sng(;iyk%w$JU=2 z9M%!A9k)d_I>FHb??f3`@)|67i@pR~S_QIBc^F?*X}XlI0hEO7T)U$p7`;#*&m9F1T- z6OAjg@0Hpj+M!wTqu+CN@Vjf#S0fK&{e3bW7t6BRXAboQ%r7}6z>Up4__ii49P^>M zZQNFU`kZu+O=l7NPkOE)1UTSL;a~I3?f7R)%;r1fSk~=KH}3 zo|L(sOY~c3zG2zj{`HV*#jmQ_5hs)fvQngkLRX?RZ4T8x!M$%m2_g)SPfhVZvsG`J zj=_2Lloui@DED*lmh_Fuo8zl(g*97rE9|S3iwY&)5!@e?c5JHVXbjui>;>{JZ^pnE z=yFvTTu*CXkk4GFb4Gd7X|8208}CU*R&TGuEgkr49fy%)W2xQ38;6p}uzF2!Y{IWe z-)DXhZ4p>}v#(D~Fz1`ZND7+Q zFGGtRDWhlM;}ct}3r2nrGj|I`1J{es*@CRq)lU?y%%iu})R7(xyI=vEf~b~2zHy@& zz#wGXfFFVQVhgt#e97!m=Uc1g2p01y`GIkGDt}&7XPlxh#^Db8eZf0uu_p zE8~v0D^L$W`v2JRA24%2KG^%5taxASP@*(QqZ;5*mL0ciD(Jh z;8M=`;M?oJE(#i>Z^Q1ttD9`w59^L%_&qc(dstobuvWCZVGaLbq$H%IMI{tO zrR4ObBo&X#E6T_TOGqe6NPOtzwg1}}oLsEzZG8UoFFcJ(?1C>G`|AoG_D0Z4aADk0q^ z>Ad-TzQ1S9nwd58JhPr>{uZ34GfIi zAO;2&0WKC)LgemH0sWzSF0Y}2f#Ji9fe{#jfpHEM1+HOWcs$0y*f7Pw5Ppw=LHQ!B z5h;p+acf6i8HM=!_wUKcX~*}ThNiaPt7{jRmof3l!cvNg8oKVU1FT(L{enP8H}8%g zT}$UnKTm%)4Ypakx{Ys4M8_u;msO;ep#6g06x5gXFAmNe%vQA5#%3hznp-Mp=mx!w z+TPiHmz*{oeb@6!_5U!+Q z5EK7?cy#qz9Q5(=+1c4CEG(RvnTd>y%mbal)tERBeHx3$$46Q5XA`p!QntgbPw ztfoy_!@$7Iu77+%-^|X=#i!(BrO|V{?7~X#z;GuwpY-fJLvypG<%a0E*N2Dg-#T*B zG9%t5d?+gYgfcLfSJg?)%#~Bo{*aa(8Ji%lqAew_{w69;3{Vk~R?^Tll~vL54+51n z^!@#VOUf#oT)mZ%dL?;94V5*+-@AWL&-~rnb8_~a9Ui&Z-ZnC~jfsle`L$?dYLosU zy`!b=WNodXsZ~);r>>znAvs-ARt-?puygjTYiL0zp?m}1SlKv5MZ_AJTDN^^cXacK ze48K+sDwqt*48(tXXS|_R3&b@XnOmHgoMY|)HV49g&UgNrf21*XXdDD8yT2d9~>Tq z$0XD?G{5u>Lh6_pKev}t)%Ngto%A6sASi5LXhc>;8>MevRPsqf&$O+-FHWzWZeL#P z{f&-#f3SA?wWY_=*{8m#&B)xYwz6&W*IrRx<<#&(LGj1u(#e~UN?@8R$pIMP9U6J) zAG{axdTtmP_fv0vF%>+56QDvIcLfy$&MF=?1=U@>T%y(FJVUorb;CQAEr*@AX}=5;es=|UXRi8{@Ejri zP&yu`yBGA-;~vdMb8F{|-kGOmr^jayi~pbg5Dr!4O};y7xcY*Dp^x}Wy4a1$Q>g_G z9yML9LnTswtutp&5`?H8p3hpF!7&@rLglFb z3KJ9KVSyS!BUfsKXhX-g8ZO4qr!59^+A5aV;=3m-w=rI%&12SSX?}>y^tvFuh4D0I z62XrT!yG!31TZirUqe>!BNhx3!w?643#0xX0Ck`NAtJ)WFpSS0ba0Ho&Hsmn#}PIT z)dDho(g21#COLw?Z}dDWCdO%j6BpTU`Wy>Oyvl9Nb+RkaFI?(Bt(ZAIDM|_( zK~MZAA1QH4%{$+WjUu6e-{z!%os2Txsuf&lDSGC1K_x(>C@s$Ai(44PN8qjEa8_%^ z7Jfrcw3#vAQ*D0gu6s`&!#(xj4{9hpA>x7NTj6U(E)|;M_mDL9=BuBW(8wpk!t|3- zggfZtJL7dOPpXZ;?NIoGMFD7vSs1DCX~pcmpCMDi?1@zzEa)=xv<=6m_CWWsfv+++ zE6AKs_%w6-V`yPmzh;27Zm!lW))A<*90a2*w(T898WK|PH#>|*gzIA(6Qaepp(&^b za={3rt9--0*XIk+doBaH+74Pl4+d!2G98vej%!6>j>%s}ac?MgS{+`6*!`Ur7Zes2 zR;3+=KTyx0ZxKyYv3Vzf;;h!pE2-)~HJ>!i25U9#r*%X1%IP72zz~5?H`*s?Rm_nn zsscGZd@~UUe(qXDOQf{zD{X)ji(v-e-tM7h_~jC`fsL@#ZaYWH9ir-*MzGdWhHgHm zkUQzr9lUTNN|v|-;s$bOVZ$6LwYhEHfwGZ6wbC7cj3lv3l|c^T5UL_XMvDF|WQLD- z$KK&{(u4%eX5hmxp?k0!lzIsX2?7rY8G{{CJMJ?~75oSZH}9 zBUuL#{UpQhQx=1*_}|6Ou(ZeE(LSSqh9EoaXvX&PL{$Sq)Sm`cf9zna^ zJeZu}ebOuV<0>&=ork}#jT8S*sKnoLegz<>LHZdJe>MXnw7%tsM{eeWhb9#rJl7?Q z$^K_Nf_-hj6TmQ5wzB(U#?yBomB$vn8W-c?515R2sx3g>oBBe(W_DCm8e%PDwL#3p z%@nFgDy2TL8@UY=(6|T&g~MA~vciu9R_himVOmu6kOoS?1y_ZMoGij1(n8Gilz=X1<9F_OjY#ow_b`XM0S`d&Z_UfX+5*l^*L zur9;xeU88H%Pl)-AQ^@T`m!&Te#ALo2OfU&b~n4iYn)Zqo7ymzmflZ&_K&?`#x zc&ye%OmOM*-p{q$l#jVP-&m+K9srE7XfdxnWK*s4LiabJdxtwS-<6oE>Cvjce)`vy zPpeuO^Ixa#SAvz@8)g(RZ`-uhrMER7&8A-yQNxz1=Tj71)We=(=^H(-4ze?vxeM@j zRlFte^DGkc8qN@T32qjI)e*`7>AkPn>Ubb(Qq5C)&YivRo=h$mR6j0%*4US-+j!*7 zTUlJ%e*g)b%6A{4i=urcRRP1V#Ew(NM>WAG4-*m_My?X+&;w?Fn`bB;<#>P0{6l0q zPRIc5bTVS}8oOO^+N>)gsPiDNMvEv{tv5$_mG3PJ8tF&Y259Sve!gxWcmfu_gGD-- zRM9BBi7b39hhjxnY~9mfnECE{T%TQVv}~ou1|vj~!Gzizb(jcub041kp##PmB!QTr zZTR9#t5j5vXBin64TlK;Px9dfXiJdN5n;tEB;` z5sqdwjrG&X_>Q#?6hK>Y2ksiO0gnJwMx%bjNsAi$QH~NM{IGbRUt^PkEr#OZmaEwZ zJFzpTKVmaIpq=^XPcU-tAi#-vy&{`NTO#XA1+oEamvM6U{`ALttHB<4!Oz(beu!Hs zC^+?~8bS-k3bBhKkCQMX3Ez-HvRcPij?tkd>GZx$ek+4LsEdt>FY!b*2|R*tkkP^A z3fbXBF}kc)DMeq$uTdk(1gu53gDi?9^%a>7b`Nm8Qzht1JmSDv+6WdueSn^$89(oG zV%}c~=tgCLe_r^o$x?K4EUSJ&#SPAq#Ff@{r%c7*jT2$OH+=u0jy4tDIlZIISA6yP z&k`OVEcTu7<$hFD^0N5Dr|UG*>mkJh-B_Ar8caq0lvXgr*ZdFRUFjMc&f)GF$rKha zro4XM*AC;#+#Ohzm;C9tS5Yqj%RPJY!mgSpC0f`;8@dQ$WB~^3Sx}2MpZ5yRAtE83 zmTvOB-4n$OIDLmlDFgYt@Zl$z#pB&Hq*h_>09?_ci-A{F1INl&+&2Gqc+WEBV*fLk zW#9crROmUrtw-J#%!Vifcjvv!FGg5cO<~2`cm5*+BykBU@FMD}_pC+qMdL6f#eK$u zccT6g{_%KJU2hfQ6RaZM1qU6O{&~h7f8HJ9Ys=n6R}fOqPbGKe_r(|1O{t3Pc#L>+ z@thIOI=Q>}NP=!*m7EwS(iLh0t?(W{=M0dD&|LR zs_JK1j%24r9Z?@16WmKhK@mc$QCR$aW~_!JkRYBWW2|dun7wwY>8w%<+LvU5Av_F8^XHz8lIG+ECNdn5{{}f> zc{Wej<|y~%SXpQ}xAt)VcG_OB29CO*Kwaiwjs$7F*I}EEofx zDTRZ=1eS%`iRR_EwN~l*WQ&|l!0DaQJYe}9Pcp76=gKP9-Cc~=prm~u_%OQ|yhU-Q z#iy_4udrth)R{6JdhQj}d&~_A*@djx7%RJ~*_T**(0v>grPr&uTjW1o%?+%qWsEDB zoP^3e^cXg0MRB_F%}@yJftZv|M4`1(Im^1+#mrAe zXzG%HBM180gi@*3co`&KSo6BL`|JKc<=gD&YiDj6vEM2&nSUmPL?!^J_{OsCr>bu0 zGt9A|oqacRXTGuxcIPd)ZU}7)LiO%P%~rJNGdKy$Y^@4YOayI#|2Q`m?Aa|~?15|w z7h2q9*6|0*!ZuQ~%`W*h_z)9rM`$3iuqEW%?l_LJYWU$i#c^{Su)(Jfa%YN*y0UCM z-+AsLR%iNGDsM)g%Ln2OU6T|SxVI>mkopb z7gy2%Hzssud<&KjvO5KA?OyCuRc-*| zy2!;L?ed{P+L{Iy)YuW0J|D2tk(4$(k99%mkfDh;5IOhCz%_*@BXxrU=0Qpvr!+-A zjD9*(JY#r1IWW1MJzkUK@n@stXjlqx#0Qp1JP}88b7dk0^Scw9*|x)9V+ts>>CrJ* za96xkcnYknXz%fHq|J%pU5UmyV&p0&G^z?LyvlabzpLifd#wl^vo$O?EHr~Z@e!22gEh0nvtmAA)UE13hVA-_pO47lTKe3>%#>@AN&2o(TCsZj5 zw$e24C=kcA*iE5?h^c8?M{M|p=uJB&G=h5RtAHAi82_34SwObWFBQ^Z<6!}@k|sQ` zi^(C}W?Yt}-WFc`{`SW%uK{g|;Qa8y)0sroJ8(PAe+ldwm#Rw#JbI(F^}aRu$}X{| zVr%=I+QqWWU#anr9}#1tI`lYID>H%vg=BZhsdvu|6P}k=t*>i3zIGS%?RiyKNUjgO zx?ry+-^mCCg^B+eweTysKT^I>q{_Wds-fO9ZkAyzP^-xnqvbBzxo-BwPU#@Y@VtOa z#U&NZ`x+P7yr)Luh_Qgz16i?Z>vWm@6eir_IaP@T&8KOzFMc61nc!!h?75vlhk+YC zS`a$C@kHS40imb8k>`sY0xnr5#^d%Hf;hL0zTJUOb8&-nEeG(05}vDTw?)ZNzMHwX zpH+^F+qMb_*KkezFziZ?=HWoQU1|@URfEQ*YuVz+{Q*O;P~|nWwJ~51%j?02yRXjR z;;iL%+2$E{Q-OA@yCzxv?(4m4%hI6}W#Y))K)|}k_)_D^)zP9x;kkpZKnHvyTU^^In?!Pgcd3&NefvZK#{e$Mm!dQPYIiT7%>7!5M5yz{p$yfQRL@LQN z@-3O$i<`T1Bbg$pH$(0oJZxb&t7X@Dl&8ZHON;@lVY?9_;$i8tA`-w!v%K-^g{_t< zfV2LqbowLUUQzkRw`r3G%Oc-l*Ll-@jEy`k+MkkLF1NM{9LOly%D(PlgM)=eH%q|x&a~5c>21}^oZ)k5tnvLt9*_{wBaI`!@)C&> zr~1zlLA`sKU%$eKqx<2FS)0#(Y-<3N?33QcWXJag>)W=}0Rwz|SEjHSYhAK#Y$`#?@; z7M>I!D6Ao@E)BHI*6Y-^7h~Oq4DyWy7)PLNaf!XpXxG+v$?dQ1z~enCBgL+81`{ce zc_$awjc+iD_$33QoP7MmaMn5w2W3;%O#_HLx|iK4cT>n=Wo)U;#lrtJ6Xey`%;K>N zb>j|uX)@2AbxnnJ0T(kb1(?=`o}W5OB;6HNynAe@UeHjt=iqDS!6O1WF)Ti#1XWNT zff)a;B)Sn~C&CgEVvf%frhX&+{+X+-GQgqJ9w!P*7nL?Tij^1Q!t0?mA~z4MEw&~`ijM&q3JhSnnVx5)!EZ~>qLMH-jx zl9k$Nj1z|8fQQe6;*}L_L7t$ox_ArW4Zs-RI*N z5(9r+VwO)&{@F}^cJy<(DPBhOlJxlJm>vvy?nlX5l}a3pg&gd5LEu3V9347g{0r(O zGTMVjw@vWR(h)|kF>YEb;LA<)#TX|&Wa6NSvsa7QOtwvCav_Ja=SP4&M3{?R-D4t( zTU)O?O=bl>{}JVcFogRSnFq6e(#|vXO2diqTGH?Bl}E#Q*v2_Zi0bJc#=WEydD!-~ z(=llfIGxamaKDUy+5wM;4HuDjXxV03>M=F3Gcc^@Xc7-I7lVa~WymZ066m-XsjiHb z%1pm-g^XmL3k=EVt`W&vuCo;XR5osi&E-t-D*AMe1FbC!v|Mb`kZuoo8Qh1ZSB*~M zK_Zhv2d6rFsHfd+=kI^XdTiUwt=;ZXIo30xam6!S15YV>d>{F4o&4AnF6?eoYH0b0BS>(r?^MxAE~kjWB=9 zUvs699|146qslPf-E1<|&3=WiZ7|U%B_*Q<(Iu`lSE66VgTRO)W^}_Yu1nUW*}#r5 z=|a(|r;gS2%;GGfK<#kRiGE2Q4UtWT?wiov z>E>AcuP^k(kmhXtZRoyZk2t#At-os7xK9TaviScp;aLkM__v~Iu<^et|kbxT{R(G4{Y#8HjdV8d0-#={$O$a`mAWm z6W?Rox;E|f;8g^pzVPGN=gZ;XpSpX}K+I#db`zVo0T&AuRg*lI887eA+R-}Fp;ea` zLcuhth&3L;E|asQ!?l{wky z>8U7T`Iwo7VQ;C&c^Jo{t*y;Yz+^86TuXl5Jz3QyoLwA#@FG$!)m#SPe?HVx;b62a ztTXn}Ej@ac0fi?Dk0a)nP0%FVNn)QLG%{vJ zBP-^x@H|ouXs#|H#qKOGvpVx~k}dTn#}mdXLCXrp+1;pGO(6xYjIt7ml%594*l!v9ADZ-Omp@l3QB$=7i6*(x} z6pazk8s;8-8h!9uwo04tE4N!hMpjI-)mI7o$C`m z3VCL4BE2%D^JVm<+AAi_?)cEIJr)uT`{jGZ2~i*NStg#KKpr4lo7FO<$c{PaUnz#J zeh3wKe>YtQ|60NW;=da!B^_|#JkMu6azl|Nnlc9!GkVX$G7Wx zintGV-E-}g`J`IA)IXDH8893u%s$Ezurj1!%52Z;>w7GAcLcd^VMcFJLdNyMRw9vv zsADbEovl?qp@--#tUvcM6Zib7EMG~Q5ajoP6wO={NbZ7{MlD&Str))3i34NUVKh1t zPi6I)la^uHu-<1dq~mr^4SulxjDof^1#)Qlse-sls+BVL9orHzS!wn+?s4Hb_y}OU ziFlhnb{2)pl}Fl0My1+)1xH1|h}0{>W)pIydGTx;qj*?j6~C9H@&9LNC7@SBFuAON zY+|{;56Ot`O?tid9hRP8rCo_w4quxJXGrVerluH&!@uEf3Ltrd-PL<2h0y+%tOJZ^ z$7nMe%WzhD4zp(2FxhMedVh9C)?{A)h%&UHBG{hr`)Ai`~eRk>)V07Di8a**eXmmq*=Bn^uCtR%74h3H&q8W}Dp5}4M! zIqxlA^IRTgYSXiED+*GCsCch~OH4gMFr1E^F;6KJG{k~z(-ycU{7gRYy+_PiL?T0@ z?^cR#&qr>0w32rW8yEUHdBMYCB>*!N~sCQAgX3#{7n~e><*`xWr+(g8f6K*%u zHeg9>vbdB&1H1E7Cq3(N7y_tJmIBC@zt?L?MxC;<0~4e1iVDLTa7E4c30@S$aZS*z zOA!I<#vkt?-?XYCmF6^QZRT{Zx)`Y`;n)|IQw*N2Q7*wKB4zR$tX6K$sl>sT+e5Pe^buTUtR zn{A%P(9@~apsd->&mbPaZpjo@TCA5ltswKST6#M97`W?KQfA+F|iT7WjB<7XbwY{5iEmUQ@sF ztO*^OSc&sM@9~1sd8YgnE16+&QF1Z| zpC2zwzq^vnef|(dCHs^t>F-t1{b23S-$&s}Qn=g+`bvRiZG ze(##|I^OoReK0ooD5__)=$X?luCNrak8YDavz2L^{xTBVw&$!-`En4R04nc-c8?NnAP~7qWWQVR z%5Yt&BVa%S&lnX#aMq$>*?xtzcks>R4sTU_tqk9qgyWC+on5a0vbrDi)?b<;QP_8Q!K= z*zm&J==oQqo%+}q-e;MW{IRmrP!KNuPMBmOEJHN(7CglLhAI4XK!&Wq#%;QfG|rbP zI8hF3_%@#}ORg_&hg+V@#cZ}6O=xnaK7~X;LpeNK|N1m`l~m*3YJ4%e~$Wln;PbSDoFfMl3OSS1qSrt zNO7u0ycxJ~%iq~Z&%2Ib3(g78lxP7-8fDX|p#MF=m;19QKREvT9?1p@&>{$1$Fy5F zSn1BV$>R+CFLF;5!*tf1WtDN16wW(Xj)}iD7o6+6+=(s;z!P}+LNd01a;5)+FYL~r z23n8URvS?QmE78T{j0@upMbx4yYKXzO3T}tEB1T(Q-w^*B4*bUqob22C!L4l76R5! zUtQ=Q-sM!8iDadXrM6;r5(p2qS<5CS%z2dMB6(?cRfg~IKt;GCpqiIo_CZIdcR+;2 z6;~3$$eOJocIY@vm={t6&aCUlBf4itl!<>+&fXyz>`sI0IM4`}S}iH}ztd-pV%3ec zc?j$EO~|pPMGRvKieTS|oxdWvtN*PjcU%{Blp1@uKxHBw`y|K8QeIzX;C7-ad}2*o zyE27h^v9QA!kFAY9N$bRRu~y3-DP8?t~K8;FKHSom6%a1X0*<;bV-(%IXg7_A8LDC zB>bM}pA-_DFkyHkHas&qT5Mv0>Bs}$Y|6ePBSRG0U>WtdYsmSjzCQ5eiX~RAUFd2; zw{m0i!(xoJKWzunZvnr=3YK}E+O@CKIoR=GSe~%vRTKPtXNSU{s@0(UO+KPmbotOo zw@e8T=OFG5ev{U7@8$P*ZH2ZJN0gziQz0ak zKKS(~Ge8lP&%^k7*>QkFpr;T!^r&54I$(Fv@Nk+D?z8eTd)|J0Gjq0qYH5c#%%xX8#h{-=TSn>5 zc?Pk>%Ld!T3dQEL1EO|Pu96&&6mWTglYzKzt$RJ^sDL@11#b6i#IGc{5h1X|1G%e( z6c9(ZPm@4aXBweC5_tSVW|T=%5*}bK_|+~NVzazvBa&{tsOjCxatB4m@1(25fn0|e z@YV!@fbH@z13sc?@D58L!OqZqd6qu@tD;J2>y4#21#6C3ejjh<8E5D&Dd% zzXPl9k59Yoe1zi9YAS8G%gRpr0N+%AOkug_6Ofy)!4GX;$J{xO)3|L2=a$itCS?i5 z*yO8E=OsU;*MItp6rk{tEl0tgBw4kd!s^zCe(SUa-Dqs=wzeK5xwP@$e82ed*d>Dl zhKb9s-4WX~0jv`=3U?D%e>eX5w$T>PJcI{lzvRtDmHUlTJsGNmC!jgz_1HFuFh zK)vCdZb$yB^)$tlKSlA}U06!&TvJw<1T8%l^JUf?C4L^fuFL5#BO{2YZaXKR=WwXL z;c77ATuxX>Ajb>xm`wb6X+9nu}07cCnnimp)}f@0@Z zw}h$1S6U3l@VU?}iB5u3Vl!s&%Km^op;_2)z#k!lz2CwRt^`NGjnV*K{36sOS0vmhbiOy2Y2iBJl9o2Rl_F*;F5cA zSR~%L#Pt~IQ*(feX^BW!c$Ra!;X>Sw;eAVY1;GSiRjP6RiO!Z03r}^uOb-7eU~C1h z&-yC#;{T;%A06>rTZg@FK0mtnn>yvZ|NB$knl?099078k_*CB`^1IkR_8rrk8QI7t zh`cQC@_5{1G=2u(7#Hj@K2IR}=q^6~aqplnNk#PdcS~-}FU4@yzX_rD-uCRH3w?Qy zBUtsc?tY1Pd?XOT>fF2~j5yjT@!D;kNJ%v|Y05Fif^WFO`jb>{ost1SD#BfA(yekP z59CIDJZxs&pviCv_yGY0LBmfdX+SIwVEn7KL7?s!7}x{HkLSN-7xqok3H?p-tusXq zBH%IYa4{}$EA$T8OBtuUrYB>~1Z?^!K9;aC#)!(+q(!F_I~1q=H9`saB7X%pr)+VN znRRoz|Mt~lC$(2)elXZ*_&*+FC}c#xm8uin{55#=5+$Bu7wn{~&7ihV6O7k07ZQbj z(t{5qD3=VMRACVp;5%6eC^Og=W*;d4*Y7{5!!>xm$)IqkiD4J!OVO_e2%?BAl17ang9fG^J-gOIiD|@@A(ML;p85mhq{F;TFc2-t$>;G$cMxj{s(? z4YOcBJs1-SKPO0g{v@g8T@+%uAm_PNFnqB~{zHzpOh^2r;P@FGdIySg&`U0_yvizQ z(JP82VaSeF})i#YPQz~ zs`DLE3O6t?6M=+-qAvp8YZdp2&Xhd00F-l2_n2aqE(@U7lF5h7qPvmz*2i8i88 zER1_Ne!nVP8rH4LUL-dzG}6!&fyh<9?-&GDzV7QSjl;$s3{auf<*p6_84DV90L?_7 zM?EDkRlu*R&-~cgT9~=FxY^K(VkJZob_JR-ipMP(cQ^P?DKetf%62Sq97*B`-uL8( z7vCdRudqu>)n#9=!)|Apxm`m)Hk9T+qBeZL;z!)@_^7*Jd$_?btt@%qdUhz_cxx`*i4rgOd`I!$lf9Otv&yeJ+cqRk6Bf<_ z686pnqhIA;Y7L?CjhRQ>VdSV<#>&>jm6hbfjAm7XZJlL(IGRhzsU#o%9kIDeYD4&c z9p*7*fyeh&xO-aI2t>;QMC>#}e0GH(E7{&E`P)O;_jm8!i4TR3xFwT^v&QuZ{1SNp zdj_f27()yM+*V8@j&4IZYiI$UVNgU!ECvby@zE^Yugd& z_)YsXs))6r?k*0x)o5bJW0q1p^8|3Dn%C5cB<@!)Q+Vk^wvUZE=oVGUm^GGWQu+$bIni$-4!E zimdS{{D?T?0{|cmwXg}iRDmBrLv3SCd-<|Mv81bo%ZczMr@I9Na>=bfiuF_tRJC9= zC-~)F2~(qjvf+-j<2bmGx(A}(JeU_x+Xj|3ufRSsd-@Mavci=4o=e4m zMwF4s#IW%*RI5HwkSYbf3=n!0AOkWKXp4R?T4CBZpxMkWhTH9v8=taPaHWA00~nC4 zZx_o57PfuJowhgMH3~nWXj{6IZApGK<}C-1)s*7sz0)HI^5RD*20Jl?Apm=93FJ5z=m)^9jy31x#wpq-Pu8m3Ot>NZ>udczbzrQiZT`vdJ-@op$*X4~ ze(5Q6=!HKjES`?HAc&2)mK*SjI)L!iQG|~O3>=1u;oi9`6~3!;zd6#}Lt{|r_5-xC`mh82cxe3%UV@MBlF ze?z%t+w$4-0X1R2B+Zh7ZL5lZr!yy06{L6r*)4>wV6cD6Zt#jXP+q$mP#A~{L2My3 z*R6lX{Uk z%^Ws2E*$4kZE&=QkC5gppf=`JguC9P8c7HcLsv`3+M6NL2% z`DXCox_`_x**eo}k9y?r8=8Fl9M&RbOy4^@K{Ns^Gw9LhtU%5Qdsquw`zCuB$>pA~ zD=^o)P%JJ-S@@xWnZQ9W7|i&w^iwET?ebvZ+U(lBZ5%jitapjT$`NA2^*LkSPnmjo zd@oe`0mZP&d}zRYBD@jVpbHk}tcz)$N)Bgn9=44@@L_D&aX4k#Mc5JEzE}IBtF`ZX zlv>+*ezC>RsY_|%ESJV-yvxow(IV`CMq&dYvJ9f|ZvBVJZf;l>r`wwayGtQWrNw`M z=3CSw-)Ya9o>6L%!$d$w#?Zsgu*Lk+N=K4mLJ-&Hyq#ex+HGrI406%i~`| z)s~!+@-@R>b<6v@H)fBeVS~J3ZTIh=N2r$dPLCv$v+ez{y(#gn=*nE&?yEkjc@6l4GiA%fZ6m1`uQkLl~zVv>}D+=dk{} zVDs+z;Jjh^g$s93rwU)dbpziO6?xq^_9tyI@191R2$_ zpuC`vNh`~N?DIZ5D0wKW%$5h%qPWCsM>^-@x8%|K>-x8&i@}`H-{0vX&Q=Kb)5Pd4 z0xKaQ;|hx1`Pm_n7N5=E>Cux5AyCpldMzpJ!Tr%``+V>i|lCy<(&mu#`)mAT&%dCKf@w2rZN z^qbLD+g%}f;f?OjO`e6HKg5jlzwI>3_SQLdt>`#Wra~qs>$o}-G|v0cyEFfDwS13p z)NUp$WjL7Seq)h&Uh1hl%5Hf>NKdhOaok{YQ*;td>+qKX${uK{&IXNpxmvrc>OZcs zlq_WFPpiz;*JcRua>}V~F4h?QAkq4`j@^U@nl6a$)sP(lPd)N+z-Ixat(U@+Ai?jw_zURL~riwW&PlAMvO zpSbtCcSP5D=x|1e+zZMkna$e<+&2Xj)!Rz*o#(np6~-=u~p~oXX5Xo>=KsK+VU`=B5wDpbF1oHvB$_-%uhn}9y6pY|46XLx4_x?Pruz-JXUKajg#{z-m^)ubM<)E`=M#IEQP;jTG=SSnqZA*do zU8!}G&dn|bYgT;!p=w8UqjEU#n>-gNOAgv#C_@fXPbiHJQaUJw?tfwh`=TW@YSNM( z%(20x#HD0xpo}_T?L-78>te4IJa4NTuGhA+UV3%q)aHiKNHiGHHOa99`AEEg@|j)g zK<@S?_`wx@zsQ7!)5p80K8zQbM3G&Sr}4T0Eglbi{EV8%R!?4GXK_FY9JvuU-nFy- zP8!X0A(QLZFK<1#^_ixb3!YhS-Fw;*JySmGXehAK@ULIcz-QLK9K)T6lD)csobpTuarG6g z*9V=Q974|1g;m|@#y_TBdiz~r)0|x;`?u9OLvy~3VLI#yuUy*{nVPv6QR4Aavzh2M zb^Mz#+fa9E`iR{e<7G@OIg<9uFP`$1+okni@A-F>wj4V%7@vFHgFwOk)WLc`Y>Ytt zHF?+-<=?4oJ`9X8X_kn!H`14+b%U4~>R9Ql=)W45iJA3&_b>tPMO#kpT*Nz(AK)Z-^7^$ zOvyvs_N8v#)S&(Pn)|Uh>-0ZQeoV+f&C?olmXw?_an^j`VX)c?gRI79i}`v^EZ_Lc z5TpES^u{$$iAG%D=kcvITVAu`u%Ycgqz++rjBdb4o4hU3^&+6bWH-%9(E8r{B++R`eQD!8m7VD2Z zgX;JFi~kfQ6YQZ4!G^2ugk`Txj|+m7-q=d`x$~Sj`OtXC3b_d`B-J(eDdWNbK+D(9rBbR)!;46&5>B7{L&K0YSy{vQWS5I0p_fs#X1|s3)Zw@)0W}?bb^X=vA3VVamtI!d_q{Xq zgGl+qBFroRvVz)^G*IIqOd*~LAUk4fV`PQL-+rfPkC!LV@JgyuKb%d`mPKNmwBbym z<3rA}bN$si*}>&*#H#SH3~}6&=vCU(;iZnG_w_|QoVN?#KYjd(ySj5Yng5J4)jN@{GHzO-evO&xch`-P6RqO8)wrU(G3N!Y$L;tOd zTGUb=N(yI-uC9Sa8iL>EJK8pg)GfKWrZ6V_9P^40>{JZzQtP+F-jox%oLSo#ydUAh@?)hTr@)pN&o-k2^}MOaIRJ_o$E`il>-_P){1K0Q1eC zjOy_5o@9a-yzEiQKxY-TBrtFbhG~ig5rqyd z2k6~(7rNCD(!s5vL;n*dL=ZaU1Osmf-1>}799IQhEvGR4|IRlH)~E(_zZ6wBb9+q7 zWB7~cwy)q1_tIC(>58JX?iUWi7*G>oBw!YdmpuRAalc~YW96!!-ea8l$Zr*i`#ABQ zUW6e%jtWU^K)NFM&@M%Gf1=(D@TO;_ysXYK3t4kh+DsH^j^>6Q#Yw7U$WDtMLzSBF zF6R`g88cN!IDV?<-FqJ_``>bF;>rR3;+g%KA(`T|o(8k1%&<%KTkhvpl4Yg89NqDt zU2{i+%23olAsqfE<=>*MzV#znS93Ef&%>caHptHh6Jp17YK1QE6`-${$=)hK*w^yr^c-y;JCF6*%YjL1^2IOD@P$V~dpb7jkA zB;ztacR!0`ygYDK8L3G<_`bH7peBRIXEpvBg-69lx|dfde;_CPzPk3qK3!r@-k(I! zq7-(-lDi83--`S0s3@MUO;8a*1O@~IB@G!#0wN$FLJvvOkdY{vAt+HYNNkcgL>cms z1`v=OB&ay#C`qD9jv_&jyv^@@zx{U4+3%d)bN1~11q@x)HPzL(?tSikp4y=H&fr?4 zx0WMZ|D70PWAy`_QWC!mE1;TNi@8K!ndir(wnJZHr>J0{5Y(fy^)b7ukA^De0egrL zt%^=YR8EPFjW~Usg_N?&CPwf1B)y3FBV{ ziK)C{3(^dlkE|xw!4*?$B(PZMXm4wqc4fceLigd{gK}%Gg}4azH+8X-nHDyjOmV14 zw4lYd_*ATa{NN!Y6*|YVG)SO}3Yo~aG+bUrx`e0Xnymrz?7zAgzKU@H@)CYE$mQ8< zvx+>XfAQ5g2lON2agZq&ZRM0uJc~1cW3`p#fV7oBazbb|W`lVems+fX<*|H2j-#P$ z)E-3ZgI%gzZYezDwyvYMXpzRFmhPqe8FjIypLsbPH_eHxgnLyK-sx+P<`F0=%QV|X za(|MrVAjX1NWrU>4Zn0oaXZnxi&LeIQaTdK5Y8ohjgkw43wu@lA| zg`>C;Za}A(Ert%U6W^TuID;#IGtN|-K9+TUm8o;Az zwt(&D0r<-&g3~GYGU&$sijygIC57h*S_Uj!DUY*?FUZe(hFgD>5nW&)ru^En{((6O4+&W5Kj6*3z z)02k@Pc%tzm5Ct$lKk(tn(d~2U?r%fUG2=Sy;F|5z-ZICmzk&WQ{(m>!=kWLK~ z?t9E<4gO7}diz2RrVGcW^cZnwt?CTka(|_W+d`9HS8ciFk*t&?8@ZrVu+m+p5~tbY z$G1*ypCk&^Bd~G(}-*2N?4 zXLG@c;G~>hPQvHHWBhZ8E_zQI4>L<1uL)zBS(q3EPCQ{n2moc8XMmOM5 zw^z8}jjLRsN+~AQj(hAeU+hO*WcBmm?})0f&f%YllHU~yUSEH)6lKQeyOrUQMnCf+ zd896>ONHmVL;Hs-WYZ2;pQ=>>)x@Qw1VKLb@u1IO9~9<1P@{tV`a(!BLILTQ{GUq2 zK#s-#o~F$Iy$dB%GWnY`^9D94+I=B3UVAlAPDr# zcN!VF|F0?`Bq1eiMN?+C;aOdncIHp6!2ZpF4kM_oJWSqss(@+=h?`=5Qy2uw+J5D_ zI+Lu1$ef2H(AC1l8~o0aqlcL0suYuMmxq1{#7Y3%2&60SPdo2i;xd6*x%>UUTQ9L>fVZ z{{Pf?yH#d9(Sim3;~q4+Il6O0?Wk8^PzOXFixdQzALQQwuX)AzAVkF^ufS^3m-nS+ z5?wE@=8dxR3cAI}IMZTCiN$f#CAVv|@+Mr9bxH*l8OkU<|PpH7-=+1Jq;7RIVci#lNv81sSQ62@x~*%)qZ- zw8NocK>X0ccD(wbIN3na;4|=lJ(nZ;UmM!b6AGR$m=3!i$eic}5bh;xfV_$h&hDt|O|| zz@OyQ$n(sly?c&48Eja(bG;`hy!!j|kF<4=*IDl6+H#9qyDNgnup zYrLBC%jlz+Q<(yHl3v=NMU4 zt7lb*0dh5gPRzF%85z0a>|i&z2i}-wc5_yDG4s++7JVI^Sn{oKc&`ae72PitADGdD zAMHSa-9H^w*RIt*VX!oMPigL*F*o> zlF^ugHrys9dOQe}x&&Auqruv^nCNZvr$Pl{-e|kd#u|BLdtzYWJ_|gK0A@HME;MPb z=KFvU>?HZuC+4_sS6_kCP_YL@Tl&{;C?lV!4u&w0;R@W90>nA3Qn#Me1%Asq{}2i? zK-}toeKxlRJ~|qTs}czQ&kN^|ThE*!q7*XZWYWwPdMpI+oogln@UHLFfxQB2DzHC} zt4<7ZqUG>$J)l`*WPiDz^1}K5)<qQvr z=i|4EL|?dK37E2n3X zL?JQnjr_OwBdBKns^4eGeeb6F1i`Le@06x;hen$JkUvK%wE8Y zDZP953?2g?NP#aLW#{eaF1K-~=AC69LR$P2wZQA0@J&e9NZyCS`>iPx17iTn z(JIe^s^}2@)Dzk%B>qhC|AoE-(b#TSque=fF!CKhzqwtx-7VzemY3 z7TP`w1zE5RuRu&Nw7u!2J2Ppv--0z*j#P4wqQr#lBJEVII>Kjij5*_O7w)wGk%YkW z;>#e8$IAq0OJl<_dMA}LLRwIl=Q)5cu%VUuPAK89Ql;CsQ(TBGAfqmwEea!s)Sf&D zVE2j2qgQxxY+m$H4;knIZS&K*xJ2;kJ9GQ#$!dQtj)o4!A_g&Ygwe!fNInb1$*^Dx zet5&2bYaoLifGWYsJ+T$5h2>S%Hl9wi*_kqjoq=Tn1IFuy)_W4_hJt11fWXJW0ZP9uBit-a~nA9cD_T_U|STCph?}bMV$^v8Qt}#%V^TNjt{vOO-BJ z%%YFpbH>}ZZ!^sB*2l)y_UKMQx;SDvyJHXQC}P*|h(TGr`w+SNW|JVaC$)PRNlE0( z0#jap^wzkgRd~C^25r-}xD?)z$7*U^Zeb%ZQytrquREAt-W}6YR`Xk8dH4y<;S*~zN?KE&H?CqS& z^t8VRsll^1XdQkE4e3Q|{&oX01oPez3y?j(xHul2nye+0s@B5r;gw zeTdDq8-Yi)fJ=q@n`s;{Ro6a;XRh)38is$~hqZm?wf!0MO>t6lVBoHQZ||?E2l;II zsa;)NyM?QTF*%&edzr(_Tm!28u0^hkbBF%;S;GNh5O^@>{ZFe6OD1(Q0zEM|7u z1?yi`(@swN(Gf$Rguq)N-QS2BzB|K_{XHY;9A!nF7^USq3{ta{$2t<#nY)?`2d=riU4LHEua5LBx9w?U>=!r0b!}Dp( z$^sOgqGd>mpBla8HFj0|%twU`fAA#YaFO$neSUxr`DH;1uJfWd`_E%7rcFDA_=41( zFjb!ggsx7z23+MWCHnLM8O);*%zMmP1&{B3I~2Dp5i5O{e9LliqlXEtA46Gq9UfkCB+%ext6}1FrpE``k;vPtW6E^DI2d#S{?!+BGoFdbY z=p~8zb;62*6n2&<(U9M~p{)+={QMnEHH5AGa))RBeT-oNeniTCE#_@c6t|#pDj(HQ zu)65>K&tV05X9^s?{cBV%(4k~V|vwgS|nhzZ$O=1?VwK$X+fh{TC$}zXK%Rho{hJA z)VkgFB^c6QO?QMWfG~tMp-Gqli^?)P;;{@TvllMG-J*GeT6jqU^S%yt%2*?+UxcHy z+!wR@?ahA;Ccew+0}mw{TqINQMYmx5w05D5x7GRuZ$>F~DDNa446UWYVHsKJ>C;^O zozT=IGEkvwRb@8FiWTwoNd9cMVF?&w4tiHeHpt=;Oy=(s|N+#US5zaWt# zbmOB^vqt$5P<1I84hIoRQ%P2H!LC1ICW!;Yu+gJ%R;&OedaqBW1eNSWmGEq^MLvRJ zxIL`MSgZjz>Im6%e-$8%rg$~g-*~Er4(+^Im=$5f4R1p&@ha&JQ>>ZZ?0v&6ocOr^ zH~G#zOhJtBDi5`)wwHWr7D0rVu#T|Kca;4*U{FYxv7ZV(%|(5!^2g`tn5E^TTf}B) zTtBq2)igZ=akqi$7Y6T8u*E;C5^S_ZO!o!VnQ13_T}&Ur8`{{H6KD;MbP~TUylH4d zHFIEfO?oU@)oL|?f+YAPJJu?O`T-C9rY8Qb&C^TwB;Y%E>c&13>wx-noPBEO(}h@q z8vHT>Kqg%$#f_b^6Df>Hf23QYvIqciV;3)Q2#6~(DfFlx-|J1)rK#3ro5XT=Ubptu z)6U98j14tCZC0_FP@GcFN|hFE-uj&$mAK@=7=x&O`)5qm0g>1|!1uDv@Y|pYX@0D@ z?rKD)cK{eq>z^$>__`p|huFG>5M#OJWSo&d0SH=#LHa$>M zrcgM#UByc~XF6Qyv0b79H16)VtX9H#uLD+uuu(V(s{o^Zw48A`I9;&k1?c_0SLjQr zo@Huc(p#3M{gCG6YHBiUu*Eo{ZFMx&z62MB@NSwF^*?ITP38;md7M%u-YVIO8!sL= zu6~ex?Vmn5;VzKDCsSk0aj96TjY%Ga!xkG&eQ7})CyWs`XY~qvZPwy<2a<$ZBBy!C zL9S8SFKYUl7%_5H7Pud2w-_ZJ&xK7;c1<4oDVYGn7*2M46QmRug1Hks0XY(X#JSGGzwb zl4`vH$!lN?8c~sUsW;s>{#Mot&;aPKG%k@h-PMd=flmo+w(UsYs^b`;>#iAv1MbE+ zhHprEg098+N)=E0uqz%2gc>Ef4RWL)HHas$469Q<<)59HiJ_kOT&sLG*G*-YQ{l}@ z_A)tu|H3VUP>-mUZ#-+N)qFEPpR61I@35<|Jt{|izPzZ8UNfsdz$4gw4@8l}2B-Z=opSAmfINEed zq}p(s9H4d?UTgvg??_a>iyu8X`HNf|_A+?~u3Hr)<>w!nU4j?C@uu|=$R%>65J)lJLkKDx?J3mLc?AA&GqWW#dqm)Qx z7OW%yO(E6L=o2|Ue!-$lg#M!Bk$0#+gqs8%Aw*m2>%DwDcpvgOhQKl94`PBZ9+S`i zT8D!PUL1CW$ZJO5zUJvuVK*h0YCPorsZ&mp5(9aFkO)=c((LXU5$sSqRaI11v7$?M zW_NuQ(FT{S))+XGuS?BH?Q+UprNaN*mD+d{e?x;o^a009DYwkAcTrsu3}j9oPbuEF zJ{^F7Du{C4@jD0{G1h%Nm2YitzP8&4fv8qAPTv>k9X!eJOK-Gv`Aab{XsEg?^@*@E z_xGw7KeW`xx3pP&r=N@nt+sE3CRz-N4S%+P!3Fk%z>;om+O)A4syfZe^Y5P3g!FZC z^Am*c-%Ul`8_`Pmhta4L{Peq+>k-~~GPSOI`);f zUy}e926hs*aGxn3#*$z-IRSZlqIW+>I9S_yf4^NTpucCQf-&#yeKB$~hh)4>IrAjl z_9LmSD%)*_q4d>fBGf`^8$*EsM0`Jsb#b=H0x~eKOtCcj;573C#c|8q7E*k;vT8)PM4ol~D)pdE_ywdYsiC9(_d8@^I0_!&1ia~ zi6glkZ&@Nfm&{$`zKZ2t;;#*>!y*?6rg;5ZhHuRVh6*{Cv=N6&W`r`Cg#zlx-kZxe zy^9^O#)dyqzUXHj1|OM-C9`3uY~3|QiatCYFn~*1dbA|(=X8%diWNy~poguTGhf%z z4UO+hE${OPX-G+I)H-#G{D%~unU}f_L%iTeew&}~l;6G(6VnpOT~EHB=^_4*m^+#~ z@4R14V{=M$Jb3IT^YWW6* zGPVEzs3oX&Kndu?oXgDo0l-4@GKkG)}yU4F1 zV;fJ4aqQAOtl0W))fxpLV&4U1>#J;h^PQmfg@LNtuH8{OAju~ls|G{+$yCK@D-yva zbB_>X9N5+s&cHthO>A)G?-N-EELq>b&VTeRMF1V^seOM&T-oX{HJpYfF#4HW_Wc6O z93_^*a#42PZ$R$fIRyuzVPe~&ToilKMLz6y6__~o&ymtaGfU606X zx>e5jT2L=}mFbvGAJy#QDB&yrG%c!=xynaxIg%TTS(l$6Qy2fl1AkQ4ZX0~Hw7?O` zQuI|(52NUuM>B#}ist^T7@w+R7zTsY%ocX1r&{N5a+D-lD&5T4-gLc(y*7wI zLvS7?q{V{$I>eGx-c34;GR-?O7tT;met2(RlNz)2kH zjp7v~D?aCbHZ^^Am+?`ft2Kgx+V^C#yZb)<6%!K)c=Do3^Kw_xdSO|HPKR;wdAV8b zZX+c!f3vR<-}cZ;4#=#>=R&uJKhAC8RVT&dtJ($#l%yI9x^O&8l`C*~(&QJV@SaxG zlMj+XN*vVizL~pT?boH3R5}dYbijg8ZUPg`1I@VEC|aT%Iz!kP81pC==j@{2nqJIZl05MS{SKVxy~& z%2BB%ph${vS2l)ZeC%dC3`rh(Ku9*%-ZozrO^#`$Ijty&Bl&mVA0)8Tnj`vAB{mJm zWZcT`sIE1~L~69|jfZ8Nx#3_sXtWw4jPp|ax)$}?6k3p+}Q z)=s1u+Uw<0tG{Qqd{RP_S}o%8rZj^E>)gK7Y?BybD&cd8Iq?n+PRq5hUtFkp@}!Y@ zC_ibQt6V!}-N9mYW!o|(1;_hRg|Dj{Ga9$jJ(}Co$P5D($QA)n#e?@>1Cc{O;oUfR z^-~`ah-vd`=EGK;fFX%J4ZVx)T0S)&O>?X-wF%M}srK4g(`Zb(pmvW`l~J zF{`#dHd*3QCS8}35TcNF{_=Zivd--uFT&Ba^yjT$QbOq?VGysf%KFapws>|+l;I83(`$zc{<|9_$w&U7huiN`}?Y?#&fV=g&O5$nM*(Hd|cE?QAx>k zv89<8HY#wSv`GCuG=xw=^ri!K6638wCU*W8xS6?P|v!bjfQmRQ7JmCglu9wPU61@Fl8 zrRm$Qb}dw!ro~1s?GMR_Y`ooVdbec7cS}%Dmt?sLj>%zVEF4W|v882!zfn|^$Cq8J za2;84+NDJHKsXr>QrrML^695;{1Dj_mGo4JKm*k-O2rod`vZ->beY^jj11d;-@w=G zqv58B$_|Ttq19W5mI_B}dr+t%BlWN~Zuy}ltRjLuBL(#|v83?gCQg=m!<}}uS`R7F z*CC}Kdd&NpABQ*ll-uPO%8MuRHI~(GmUg5S7@FAcN)b!ilGD5fTQzzG-WwL!3n>)a z_7$IA2lAHxikGfGM|C><5to)Q#q>wnHQK4f5=qC+!As3_ zX;b_Ank#S}xib8K=aF8GK zcRjnzt8;W!98-P&Zl)5=9XL0(YCPkRQ*Cb-kLZ&#gk=bTAe7e1(F98PusrBK-M-Ra z)cLkD5G@x94&Au*WQs3d4%YW{ zx0BbVhMX0i9Rx~72uqg?fd0iyc{_Mp9WC3;Mmi_;HP!1g2QNJCnE?@`ZfeCGuZL9+ z;a|~T9Z6u@A%1xL=x$@DR^2B#z~nkGB9RCivd7eWxVYFvEr%EI4UX*Xz5CVlmJ6$V zuR%!Bjjvx_TQO$>nju)#b-kM+8V~zcrQFqlhy|pvyE2S_P^V%zg6Pj@net`JP%!Ay z#iY&bb?;WY5~EP>p1#%6z_Dq(f?k5foG|6)LrpZOzCQacv}V`>4eAW-<|BaPcdqx` zWke`pofrADZt}~OLV*E0y$gvRn**wwQxEW|^Yqe)(*<_jS-rx)V?vXDXM1v=Y|32v z%9eO-ceQN`>S3l7_eBf$Q`2T1Jj{*Kwl3cEDLm}zvNv2!lWKZiv)2bPjm9CheDYg4 z6+E(lGXeDLzbq!C1rMd{pb)bG5<8Umw_S!F$G;6aq<4Igz5qLKK;roSHpj9+#$^cB zXMsO}9sn_v5c$sz|9%gc+oyMD`={Bf+KUy$0#_Ew=*$bz^;Eff;(T5y%lgkh5B+!& zNOWfJK6N4E-K=i&f5`=jHD3Z?xG*FkD|klYz?>K zmh}|KM1JiSTG%LzP0rmBJtUiaBF#_$8@MGi9Mc_3zbdt_>S_|1= zBJDcq*rtdkykS&E-04EYdE5=WnL=w4$Ig&qGSqpLN3VHes9fXF-FyY%f}2P=20$SB zCNez60#k9zDQe~N4H=;}4yC@%Hr-wh{q~b*l0tp`*L!Zjbh*e9K&nM~Va==mOdAAi zw+LD0z+tIB&9#c*8D!eOseTk2z5;sFn94$%EMXF`{m2o!1>0zIEwNNqy&1Dk9OpE&2RrRHr$-mm0)^W zjpU#x7vbU^_WPER+)F)?;N-4aGiL$Ch}sK2Y`w-KSN!W8rF$Y!K^g2YSXhj7<#YaE zuOs)>jmsCy24)EVT%i##szeg%NNo#A{l2AzZQ?9g+o+Va6eNmtDnP#?ZryXa#4+_W z9@3cJEA&F`VZ?q)l%vq9fmT>v_O-${Pe5+GgFFfm8kVrI;VA*j@6zA4fQ7|xe5mez zrM+$#d`Z9=*p2XmDmexISsim9+2d6-7mHa88TF@PloAvxj#TJb1(IxTkCuXPFA*5OAlw;U zxL1pY#M1a|H{a?xxhCFT4i6LwgHmu%Y_3g-hDagIe=J3@JjJX;H+qO!+cY0!9YUfa zN{04c~B?mE1jG!(^DdzKf4(g|xca6cWLX4T1S#FhC$ zlfN6m*M75`8aRKbW@f6=x(kwFU#MSkJElfm&~j_{z+C5^0xGW1DQ3;vP{GXp;&_UB zqf^>W`pMXX=`+8{Yv8qN!T*`Z45?%eD?qh2v8(6dl#1%5Uaz%#p3K$#$YBTq8zawJ zLYm24-Vfxjmu+o4_kA<)eAVSRWyTbG(Q|HG9oSkwJ;G2PFpGcjXB}&qjkbYn{*(1P za+eciqMZ9VwI$_qAl>10F!b>GqnmRdqn%sz2VX;tKMljzydBh<&ije`CAd3?x0(dU zX5~t+jF3W>L5dIC{5&#(x%P|PB!XGy=JQRDW0MfE3s*Fg201{u{WoLGP-(m0(uDwd z!LNlFoEjHpIP z**$Z{y`^9R`JDqVWjjIkDi{J017+)J$yc+L?#M1%m_sE7JYE{WDc!FR@r>QeL#Nk; zVDE`82FL|p&54`YAN1YwC!;2WimcQ=Il$MsNUP_p2H6rFAbolBiPWu>%pJBgvXQ_q= z1s5XtWLmbu-q3huoK`uP_fe`OPm9P zLsNk2@y#=Nyx&4(HjNi_6kyk=M4i7D^L3Ggx$LPd=6Md->Bg(6C7z{E1Jh@!vbTTF z8MVSEY0Amk79qOjUQ6_WN8(b*as6f5Z#jkK(%Y)OF)x9Us-=o3`t2bG1tP+a-c4-S z%M>%U>O4%?MKXfr?HF+1rv%?S2iqAX5RbD3_QovhGPE%r&moQq!Qi160+7Zo&VQbq zXMDv6BT%2Gfi*97k-@epS^xD{ssBpti|*~8n5nEA1dMTtz#jIuF*7`xB2=-_k8rTT z3jl~`Urs$(^k(M&OCTPj2j%JyzOl&h-^a-s zv%c&Ak%pg+d&$Nz1;(#uk{CO%IY1ml-f^4A5L?c{vzF z5mW`mdZ%f>MD9|;9(l7f`T1p`P)+NK)!eh^UJ0NKRor)9Gr!M06kimd_cZ@%J$2p( zBbbeh!lKfcK#K3a2To;|cbqFtU$%;4eb}W$3@#C5bX5)1o!$2hFsR0AG&#rvb1h!S zo5yn%Pj~kpYB4{5PBkRYldOUF=f@t;{Zx#Tu-z-7%xYpwVQxLh(GAv=IWqXgCBteh z{bQLTY~MftE?>()VC!w80hNFJi$@*s#^v{!uP3ngFGotG;IJemkM9TVb{GP8KL0r)I#cZv z#%;w%M=oQCW(pN>g{?359XN|*B|H85*QDm;85%T@-14{3{b2Bd`DliHf*hE#?b`a$6qvE+7Js%;TY zBh*z{9bkT;BcJn?xM}@(B1px#;k>KxUAe{r;Z{_~R(v26sIhq7;n+l69XxZv_qDhn z=je3+{>hgAP_WT`;7ap5kV(4tX|XK8yxR854WlUWg!%BNe7{kk>}VbF&N)|Q0GB^| z{neDs%DqhP`!cs7n#nLk8~Qu59JtxTzo49xm$R~3HI?P18v57+2Dxgu*je9&8EX3uesL&E%v zCwrJ^qE{GMWpq`3s%oJ@aT8$uJ6#-8^h;}=O4kEc>Au@{s36wwllS`fOIWZJqdSUt z_*dne1MUZ>*|NY^%Wlr>JoQ)DAF=p9mBev8DSJ7Z&BQXNd(t|61Sw1)CG{rJ?eCH3BirRAJueDCaM$iZmY7&=bIn`PnNR&gxF z%5pC8S^)Y$Cg8ln>!fz=%3<~wwc3@zYqHDF=4mN3UKvRdtXw^6mT15dIr|W0-!<4> z$+ICPyL?Gj|GmArsK%>TLj)_NKGF}H;jf@`_*O`dnjV(YUmYOMcCg#`I_SGHOhP!w zVIGWt&V@AZwkHTWe&_$KUO4@yyy)!ZLH?%23D2qs$kyfhz=q7W<@ey8SY81q?yIETik5_A^C7 zI~bfGXw2=$Sx4+~uA!eFGrtXiI~%lb`P$q%eYs`!QFD~Qo#T{_$@dGeB@-S&u%dS~ z-t-LnlDYhoJVa})^jvx+HPnGiAsif4qHdV=+`q@9xX9lAALj2xJY9d|F#Lznl`sYsjE@vouo=!VHdmo@Y&MQ4##` zJ~}3cPTS2q^+`C?HC*{5<8FH8|K$OvlHfUWt`d_D#hJP!bYBj4gn_%&Gk0rAOIK^? z2Z6AVun5191i$bD9bsWf;ro*J?%fpD}f~ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png new file mode 100644 index 0000000000000000000000000000000000000000..b3f6e926b032f2d79c7be1bc24cd4fb2c9caf6a7 GIT binary patch literal 146525 zcmeFZWmJ@J+c!#zf(RBUtw=~W(jY1V64H%ycekj3N-6@<-QCTIh)Czq-8J+K48y=a zNBuv~z4v?Hdw+R9?DejPwPxn>y3RiOIDVHOloe%&2`LD%u&{__pFdH-!on}W!ovG? z`4YH;RylZsg>_ZZN?KZ3R$7`)+1bI|%GL}E>-mQmO#&^|PO=o;&ry<=*mzm*X7N5z zNXlJie|^Nka_3HACt|cu8V{k?}p;vHq&Ptj}d&Bw$*ab zjBKetoD1kXTkEOkJ)XsuggugZ(Df9G{p%*#dy_Zl4cX3zu0yJlc&Cz->(~U>1PyFH8^kNuV^pQTq#>(yC!#-lynb5IZJks<59)t?RC!Hj_@z7kJRoN& zAg)QTGm1oBOTFU9J?jlMg1`{(a zD!$7lS337)C!Fte07I{h1psABQ)GO~5D zo=W@7kz<^pCF-t3R$IRkA#DOtBrPd(st6*CGb~bl-Iw(CNyo2_#al1!zkH_=Tv@v$ z@1^=)|A5Hkw#oiGoI1LdQ_tW$Iu=D5CW9}%SHeGcyye7Qc#V&PMg9Dh0ztHt0I4g2 zAJHn}r9?W9#r+G<_f_eu;AaoIyx;9n5PkfXO1dv~zV|3Vo0Im|-BOo>hO&EpebV9CFx8v4fnD*b3P`d_{?q*&|bXdrdW;sO+|RnrjM)c$L+Gf?Ysy+k3gPbAzw> zwfXefe)X`V@h*@DDp?TKU&~xqj8N>udJf?{7#mBpobbRM+&Jm$y}-rV9c^uGZJVnf z(7-~}x54Jlk}J)>DTVN0StQoqa7s^V%O=P1mCtKU&LIeUeYY(5&8_`LiBq`y2 zDXQm}ZWx8ta5a@k zJh4w(MfUDWk=}Ux)bc%zahMjJ`}>>UC4HkLA5eXe`uvIS!OiRUZd$$ph% zW>C9PbN$+N#n$E#X-}C4H{9N9Jn^U_kNz<6G%*Pm(7{)^}=tNgopox^N{(-~eWhu4WS;|I0)IPApjSYfw{WkU1J zvfD?yMp?!ti=p|@(&BtrMsk{%LZcii&%O3qj!AyoXeU&=pwe)yp)dD(F#T8h6MC}n zJX$BOA3igm8vGl4XLx4B8pd2DHU{B1aOb1gE&k1!;SRVRJQJR~=`bLXJtwGQlQs5y z=U2ril+3JvB(W{A@m2FXw%qGncewa?t_s|`Tf#4DHExk*vTX6iu4-ug>&sH^-ti8t zS4~58&=Vp3#J;Sxp|OFf4K(8=#(Ru6qI(&W8B-OCUo0tvDja{E<<0!c-NVzPkx(5E zPozwYE~i|c8poZ3QqU4ZVQbD4co%|g~El6qiQyWTMtM1 zM>Xx<3tUa4N`wo<*}l}(oMM||nF@8!OHNGwX5Xwo#{8W*^5yrJ{F(OSh2y+1BG}e8 z?W6Ta??i5i@Ci3j*@+9A9UTl>JPUAfb?b4P-t@ExTm3q~-EN}uv&p@}ZBbZVNZ&0- zm_VrC$==z2#dz(z$HctzaLlT3pIe|!;+CeMhhVvGvu~j?6!qK9tmoHw(VfC$ck9ML3edVx4 zZ7+ACDo?J3{oXY7A@ia0D%xOMOYh~GyZ_=@?2We1^$d~zSI)Cb)T-6=yiEef-h>nF z2fYbO4oxC2);}r^({yi{E^n$h>o(q5JcL9b-Fi#k%Jam!a=7U>7~OS>cbtH)v~+u@ zS*qPq7uDgmf7&K@+Eo$bpsAqaRx*_Byky;F-MP8LzhILvnoz1=gXnH3avLi+D@3SK zs&N+hLsDwb4=zE7;grOJu?}ZdvSk~qtMk3Y{IUEoy6<%fYegp#%HTZ?VRP2C8xE{< z2)9N@pW}$H)UfKE>S_^UcaCD|tp{$+jVd1clhkF6`LiwlhcwAzD?Y6TowIAGA%y-M zZ4Cs5wIqp_3VjO=u?*>=hzi~yMu%&M&nR>%==Y}g+OZq3hg=h(n)Uy&{2clcmMI_k zG9reG(w5zt`VoiltjGPsADK0>b{>k1$xV|qGqYYS+anXhQn^YgMk!WeTYiXNLj5ki zYj@WuIqUQ-*BxPxjvBXiLxg99DLEPpZ5x)I^0RGh$I81~;*|8R)H$EBpf-bSu8%d8 zbQnUs=;6yT#Z2#`iz%@Au^XP*RdmLO(vOSS{=w zi*+!X+qt$%eC^{SR?-K@K?PqbN>jYIkrpRNx#t*@`mZM}$5L{pCFdJ3+u1=^Sh&CM<6vQhSYhG)ZKDW&F@I6ugX#0<7x!~876JIz4e;?y!}+H*enA@UKi7D_ zz%wi&8b(aN~;Ob8QzaEHVbn2U}L^xvH7z&p1!aTuhv;99^v(?CCJ$8W}sdxr*Jt zkD2Jd{{F1f%+u<>XR>$syDhLmPRtWdZVoQa{~8-~6~){YR<`mqv((v^>&>tDP6%cK8iS9KRNXK4pJFsZBfe-rF)=YM_px1%U0 zX7B%U7Jm-<_g!$(;)J4{|3x)%!sQQY?SPO}R!@}Fz%O7k%pZ;^_+tL^i@Dx#W_|l^ z9}7zoOZJJBnkV-5%$2j7?dXer?CVmbxACi)E!FWzp4e>pRoAGGSXURri#}#Zlp7b5 z7r(Dn*Huy*krVqk+|Gqn^ zWLgOhUcpvZ|Ih3HX^P8ght&V+8EIZ2R^VpCL8a5bT^(lGI8a}L|Gaz_$+U)&Gh3?v zw7l1n{B!^58CGBeA!%L-s|2OYe>fdduyv3B^z47O{y$Xzw_N)_4F7MB<$q-T|6R?o zpHrU%T;|Hd!wdw*=8WHzvBtTElcyz9k=wzHLANw+rKuNYn@5*e3tBE6&~Jnj)B&bh4} z-PqFoYt5pP+VAB89F4EVcX1PnlF^MQau9wZZd0!Be-^Zl$~~L8;u{D*JmeIC@Ej{4 zx9&@0BP@}Ge&SZ1QwJ-%hV96DVzNF6SX5h1iht?!2&sIBzLUmM*U0 zuGEy?t`=lCZ(;2!eS(gDATe1w4T;?I-PE0{xG)$Ae0|svw6^qVmkAcvHZSp4Z1XAr zPqDle&~+cAu^RLn>$DJ3ts;=gKLUItT99nfe4wh`#qQCJD4#^dW%CrWX$x_z(5$h= z)K5S{@e17V{f|ak9Cf8%cPBSKIW7+vq$XShs8zE-Tdou zs9_AbN_|*S+!TE;S(m%yew*J(L`nO{hO*p^H}H|+wW*Rep;B?knXm7`y6=pEHe2fY z8NQe?xh-RZ|55XS2f4MsG=l+AID9=?=8(S#Q=Sd>rNnXXK=rIw9U}jnfH0M^0_P(QSt8K zXkqN(@iF^5790`;%w~Fymi%;-%{&8LA&Ee#<)s`M)-oH_2h0H$ywJ<^KliJT2#N^p zr}y?{>Evu^>E}Jwfg$c$`CIL#cBTB_w!f%KLUiBv%xlt|;o`%FYA1j$X6zqR?j-ppCrq}k)K(kyl&RPdZ zoUVRB9T!*0Ze5)$ty!~KN!e9z6dT!RJx1y8IF)Gn$M}+0E9UQ-)SGb}?`9{RSwdXZ z{%f@=z%`5MGBa&1lSyEYq?DeF*r1GHx9wOaBJ_1P5o?I55aUOfV2O}ic(p!s0ji?> zV@SZ%%D~E+QoWx)%~Yf;8a7`26j5sDPc&v&1|d2$sGhFc}uyV)oE`D~Cyl;*^o{64yn4YIinNO@j&D!45(rR>`SpFZ1y zAIqRtAc9Tz|LP3}jNW+JJs{jLusc?27%cbXC7^ELDtVfHghcS2cFIpUKrueoecd2r z=TsB&+u@e1!Mm(xvE|9gLCh5Wp;HUvYv%DSsY+ANJ{RffIC`mZ9dOQt#D zS*r^QDqfkm`H!y32D(aTQ$T7KDjGl1j-K0T99lM9mbr_V+7#SbI@jH$Oz!o;G!A?( z$-lMwye8gTSAl-A^o1Wk^HrS7BQMhu++3(bTXX)#FKLRHcm{8dIkEU>r>PM6335gK zn|HJNB}#9dweG#UfHqt?6mBk~)P+V*P%&_aD=X5N<1Tn9nA_ZMOY+z3QOJN3U!n?a z*p~HCV(ggl2trPwH8q-c3AKhB8;r#@o?r!r5mwdO_c6;WmQUJ%q1rkLFvrY-11nA} zthj0V=5I8nzzS5O6^Xu^dy^PbbT-EURMmU0Yb)WRNvL{RnU19$iAt94-`u29bd~~x zUarcu@4GBz%ROoG1a-Sx9tSh%{XoP>?`^xN7;q4n71F7KUGlm;iYCt)itaQfM2@k*&BS7M$D;Os}<>Ipu5gJf^Xb^EeXSCDNeM3SlIp(5hh51 zo$^v;Q|y>9FhvJ}<-OGb{6h{2na`GnzL} z%t-GBj9T~c?`2^HJ|m}7o>`a4`-#%*>@+BP}6KjHN()B;D{n zkM=?*SnB`A$C4Ttab0bpYDSyRbSO~V-~*8!>U3Y%@2n}iRF6BEncZ}a>;ml0*K zIbx^8ds|>1iIKCyxmwxYnGcPQ9z(R0b)~HJw^0u{lu|itN)3JYpALq$t;Fi~6*V1n ztrAF_1R~FmA%@i%5csIBd1kl(&jo7k;=8EeoUhB_{9BUX)GfIA>Cr|GaAEBa8(}qx z@mxla7xlgN+sS*g;|#1*{1DoQeImPGTl`Q7=V(-7^YQGI@2=0Bd=f1yT~sWfg-|it zIt%uIKLcHGwjNF^G5aFvQMtk;{RUSjc-0$-$WC46%e|VxoXkk^$t1T?ZTngp{}ZFi zjhw{Q?ZqS^mmY>VJ>wM5#VfpalYE&x`N>cSwflIwLVOEqW;a1_*0U{7KZV~$L(!o3 zRO-@9gPThuq*q|-tItk-M{~ete(p5M#Az9EwOZ;^|GiO)v<|xOCtWr9-6@Eppn4etV^^>-3dHW=rlF83-!C6bNI^N2Y#ZXG>5bqa} z;-`6Du${wC)ft}as zpWTvIi9=@NKs{#M;wBs*=xLJpAa{E(kih0ASOA}Tf(D^lVe+Kgm_BuQ3$8_~pZh&V z@-t1Jj15@roPgiyk<;#&u7lWN>IK{O`+4(Kw4md>Wac~~GT>~+F3ja{2R>oCP@01jL-T;jzxT z1Z@wZyKincDb_d7m`}Obz)IJlIqkRsr+8DJ&*bM-B~#aZQLh8j;cA;PM*{|jk;08( zB?&jNU$Z{DV#Cco$8r_74cusL-6z>6SC~>`)*W>{S#8EVpHOoSQ}yclqGK zjvdTOby?C6KCBkPsuoQdT6UcF}Exe%1(24kNJB!2$RYFQ_;^V-THqV-b2 zR=HQL!c?KCYZ*ul!er=ePg+nrg_eRW`keMAvYV@QO%)BIf6fa(gx23vOH&W-1nB zC#@;JU0dTvCH6~Ph_V$@Thp^eG%v9vIRq44UR#5dV_$G3!(LKz)bz()Ufc4ykvy_G zzF_=)MFKU^|Ggcf?jFq{@~*8N=9Ech!<4hPZqJ4WxkQij5{)fV|K&`&GRfC8P=55? zw0&Nsp{?}KG*&B@2Qs|_RjOpFgXcWYk!w?X$zH4Rt6xnboDSiRt%ap64GIZy2Hu_1 zP94-GaV6gV8>6~#icm!g2Q?B!wMb)y&3kJOW3&D!*v!f2+8Ou6eN>iL!&9B+E?bLRAfEQ$gNno}27`5c*w{(vll;3N4+s@Hz)th%gQ*$ZOH(Dpm&F zk{5%SW`zi^Dn*JRzEX~<9lE{XBdhVY-*R6wi_;%H+Y6ECxl4jN-757ELySo8qmSo2 z+M4woTk$!j49^eZ?7CDVU|XdS8ie_Y;Aw!N;u7%LRo3T6%r+QD*^2tXn@XWp(ZD<I%I>Hl+Lk`F&GQDTq#)y63*0$Ew zaf4Q~ON*vS=+w(;1tQ@&SLYZ>p(@#trJ+kNzdnV}OPX=+!nPyGIlQ#Z zcZA3$l)dJ198&ZP#IH^z&A@7~^O6Iu2#s0dkxqZaC#s$lF_9NEu$N+{zoI0Y3;fQ{ zp$2*c0%=qwnQ*)L$!uM#?3hX0Ii{*wR|0``pfKjkQeg&eR2j`}XEjIGAzhcNM1`$V zlrm>6+>=K4Dp#kzNjaWwpd#)Ef8YJPb{pTG4EpityLmy^J$>sYnII! zBDxvTo3};O)cfYb&h2fLt9$i1b^6bhDaPrn6ORh!pu<7|DkS!B?}&4S&E`o?%IRxs zGCpdBz55$55(T7oL%;iY@wf9kcgJa;x2?3uxb4n1nsKaGfcQ9n=8vumoIAXX>%)I5tpQE#^ z9gdb0iFC$+BGMHXK+5V>k)PO781Hwp2IWdIFI-ewi(?AxHZ_?n`8vf;o8gG{$(0%R z!hjmV&+==yQ1S49>gM`s=cy?+<-OtlG1x4j4lCEz+Ju4af`Hxb#y7-Q=Wbz-9V%O( zlv?kFP~GSo79LD<;!}nlZp}dgUilb=AZ^a22&iw_OH1LWj*5SMm}Iz<$k}popJ~s& zquZ6yU38Soa7vYZtp;J4rZW{=4~wB1H|Sju7N)^CUsK_Jh>4Dsjkf}XO`bke^fTGA z)|A9*Hng`Mm5a^cP91Vh6ZC2dD+qH;=Db19%FmJahF)^kTMWC%#flN!E6sibl}n|~ z8ui927{16D@Xxj|P)wVgb-QuS{*Z2y3EvJ)^8QqJ!ZI@)r&{QIS3&9t>Xmk?Xj!vo z8?ByQUp3b;`HXgi?^bNT(uUfF)%b@qT6-Avg9Uev`sL4#)KkNPcLeSA zUimPEbcZ(Y2UyWuTwYc!kYo-i6%L3o_*JjDCU3CmYV^ZIbDsv~7Nj}Cn#YCS|Z&i3;Q z`FGrSX*yH_t@TBfk{)p@r;QpGJ@X;RKTd$qX?`1wyq+%#b@hVHhCT94#8`oWhGftSLuiw$fimEfC0;+8{oIsS8*QerW^$Cp*_ z`u6Vva*`(?YStWS-O4u(RZ0aUX1vyt7tVlHnCs_2shb#``evGZW_e8w^IOBk?|MX$ zr@`rgkIOef;EaFD-MS`d-g8HA?n2W#Y%{B7P_p_Bi8ynKU4BUQ909&>#!VA7&VgUM zF{{l7KH6l2gv@qFC@Q#JKMI6YJ~hyyEymxm#Pc}dh?a;W#!cG1pTUA?GbgV(+`9Dm zcG76%hlCLqMAfh&(O7w#%R7AhMYD9MqAKz?xj} zJiAZ+Rjq95LP2Iq69021ymlzB_Y`u0f_%GHetq@R1tSHOMm;*KhV^vG@CbQXw)+ch zp2mz_$kyfUfjdTt1@m>BV-n~4kz8CQ2dKk9%iVQLF2gl^chBp!Th3)|8bMy!xAQba zXwc=TS_ry0*R{FTZ=#DjQ>Py>Abk31JXMCdjfk^}i%mD*sirMlaDsBOlu_i>HpgG8 z{{R)v2!<~AH(BJT$Xcb^T1_{92WuQCQSfG&$)5GcHRaK372gS85Q!z(w;f)T3fU(G(FZR>_pA`7 zb&&R*U$8h}^Kqo=QHLTid?EL@%$jY7{MbW2B;9&D7!-^hdaZM1N+zHPFo}iaNzruS zq@&@E4t{}~;OC3k6GXp*s#qgDGdMNF&Z0hWbKU7fO7wb%Nr3-#^z}9eRw9sA)PD5w z%Rjl4Cagef&cTdQAfoMG=2#U{S>x9gk_*XTFJYXs2ZnCk#VM-DXj)Zmu@1%;UrCu? zZC&X5z^Vg-CzTC0pRVLO<3h!}kM;`Rl{V}h)(%^5zksk`6wEsL&p|Uaqy3^(>S%$B zxD;;yf(aq=+=WZqi9jT+4w;B=F4iviL=}$Jrsf(r$>vzTlB~T-o^Xp)`JAxiMA99I zJNJ*w;Q$ciI|f{PH2-=%&!>|PNr|8kTA z{aGQcl3|G3h-LM*^Z**-o4_9@u3D|NdKrwx`ze2{YPc0A)j z5{H^y@PEWYO!8SGe`X&=4GRKLFA3jsWGN!>r;*R*ap7ml>K0mYhw!%IrJVhCzg_=S z4-G^4YSmeWQ#n)&)B$JZPu408iGNv5@PgY`^9oD-0OtrRyOq-V*{eWr4W`~qeSC*+ zF@U-B!?qTlou}@;EXK0(-!+F-3YR zC26^aG{>7h=LHJGB;nbU)s|_(lP0HwGPHJ;BrA9YnXJuc2i-dy3;_h;2$56VSNW1| zu3tZZR#x$gIM6}bw@Z3%nyOVDRS|~gU!JAuNQ9m4DjY>HqrW9dsNcvNRc34pCbl{B z8?NL&Li?>HdmIQx*Y{T$#*1X#y8Jt18W;j%ark<-)~(;7?&iH1HHUA*#d4r1uMTPtw$t4KX%gZ}JCN-FzWi0rOKyV()`{M) z)Ln=0D>>#pCd}gKgx&KvYA#MXB!XpGuc+CW;d6b7^Zt1|k&mMWhwwC)W-Y;0z6>%i zvH#;HQORb<@JR7+L)n-bOwx^^2CsmKrsSgq8RcqQfP^)5)@X#Lj-aOAI-_2>3dkcb z6NpgcjUHY)%bKqU{x&cdJbCW|H^s2Tg7E~BXjeN^W2(D=3ner^b>q`}M{}!)fXDj9 z63BTqm<>w0@jZ%@s0u-3DyQkRywiqzlv3MEPYvn67+e!PInwsYMt4nK&ldd?`B$ZYT-2z@st`lT z(PP8-TaDwI^V(uL)zcTOBX1j~=iU3RS)54cOIH6p7_s}=Fh107eolJx{LPEz=tCbp za(Wx3G#wM!)6f@UuH{3r+ZT%KlGZKrm=jQ@EVu;igbA>n(xgBi=@xyY5^9X6eA4qublhq zQ2c_J?$nF3wKB1Bmwqu99u3Fg+S@_F#a*(t63mX<0iMvZ=ibW3u0v7Rx;QEF&Zp+i z$?T=uWLK1e0JDam$S5~uTXiLZYl3GEREGeK%{~&TZC!h?tFyzUNg<)PwQkgbR4tdR z#%c5yIZdw_Uk)f*2iaW3&&I$=97vq8Ty%3ZvRO1EfN0#c(kfejd!Nk%7rRG5@@gKWps%iNDe%?# z4M-6Lb+Ztwp+C%7J7IDt$du z74T(5qQoWdN9IQ|AF=|itNTx{ZO(1vrqaydd)T6R`Hw^+HR3HHIxEjr75~J=6-*#w z=+nXX^24N0dTIEEvBU;G#9Pbe4rxvhjuUWux^zH9`km-8{u^{wN?XfX9ET2nN)Css zMPJeeO+{9yq`vL5RL{9t$!hNoB+-<*rapfiAlnQeyiJ5cQX_3l z0?TQ#$|`U-zv#?2a!(Ey+E5l@5XE{q0BFPpU4OTQ9GVe-2mvw07Z)erUjDIn-A7Ts zEK+3FyHg4T$uM(G5It|IR#GAi{WWXT$Sl~Nh9uC$EgyrlFu^5wg z;aLbXRV38l;W!GUxG!0oRKIBT-@QfLR4A7C$SFfaX5h}~Ag#<{$fXb0#Byp3VA0+> z-N$c;I4+;@^0|^n2U~tn1i5Cp<3d9;3!60GH!5N^lDj+)>OhU+<9T#xSKjP2OJ&M_ zy0GryveQoeTJY2_T4DBvQf)saV7uw%>;Gm(N`TJWrkSdc(+DT>gQPoioLo`%DXu96 zDzkm=ffivAG0@qL!FKS19i*-ejbsB{F-{NMxPSMOYk_-@UV{f>rxwmOz; zVR22@ghW7NnW5n6d^;e>WGsi?ZA%1_%v-%*6d!uX1anD0O2{?P-pnRT&bWk>u3(;D zI24|Ljhj)3KS)=ubT}l>8Dy1#JY5+a%L7tr4L68f%|ABjh#)==6olW*GoR6esEoLG zL-_z9{ffY8QXJ0_II2fJ*{$#be(ratqQfmpKaDhAiWi}?M$xI_lRumHz7s7dvOY?@ z=)r~9yZQrTD!IA~SW3Vx&9EQB^1fK0U8yAryhS+bE!(Z8n5NU5HwFT|3{>g2{RxXy zn8#G}vt7o6mEl4fs*Mo6P7*bns}5qXxf8@Mn!TTqd)G+orRmtETo^r`!$mWvKcvX( zo6|VAK-Z9k{k9u&hiwiY-#Agl|v1ZHZr_HR@cs+xCFQxQ)g=u z7aQJ+C|yst!TiuQRVA_N-ZIL`3ruB*_+nwWkg5T$BX%%CIjIEF_pBshb+XMHuH_WX zWzhCb?hdNqpKT7Br6Ap!@3Z&WP)V$cZ~CKm%8Jt9YiHmDMtGdJ2l&r7>0u;{B@~C5 z(DI~WRpN{X(&VcslNRgdBFvD7*08p^W!G4TV!8}SB0M?h{hZ_^RFSZ0(>~VHuZWg~ zU(tzgzQ}DvG0FF!b0O+xL86+2&6yS47U34N*|DCr5PItl+%7b@;ACdkpFE6KLR^oD zA`h`O3o1PJ60sn4$(3A_F7l~Q^zem;X1G$}m>drD53x`gw3ihhExe;^*k!5h!?kbN zU}WfwANffGjO>ETJvGId!7ND#?GCGia`9EZcd=(~acWh~%CCJux{oNs!}F=&9r8Sl zNlVHC#B)^!!1%e1$wydVCSfKdyflFNNx)?dot()lBNg@~GX~MTf5S_@kCFti z_hRuT)-SOsukr~zAJ53{w=cINVK#B-o@|6!=g4S=!;`N=$+cU^V-UT_$3 z83n58&F98k_nvDuX11KxVC6pD;DqswtX3uPC8`S3Ij zTMU052ul80?2X@c&;% z8m(Y$0(H<;5JOw3p7$k-B&LY@Bmm13ABank5J4?cBRibOi>Y|?8hyvB-NpEpnLCUq z=$qdHN{!)eHqtcoVj>2f0?;Yil?;Dnjv2^Ys0_bUJdnB9&_&r)pB*6=I8-xEP0Kvq zEDWaw_tI77_>z}oEk{H0=F@df=AWLDaWkd*R&0#qt7dLfpavH)lNe%QWUgHDH6qjj z2WR67&;%B}v41r|2G9g-G*CRVMN=MZjZh{y`6IbG2E_u%8sx9K#JXEKDbqY`eXWShaH9~}E2{~b; zi^AOU9<{ylC-Phav_HW*mqGBcH>=8@dS~YW)f9e<5m5fdAb&zeG$i^6aHaNXzl8k9qw{EH4HQ&<&N= zZLW;dch?@Sbuh$$Me49l2xxuh4-7!{Dcsv#CRc70d3M;}+sz5t_6}sN+hK})vu0o=DRVqb*Py206HleCc8Ct*PA)8cw`{RPbl1;?Knvmz-d=5p&RN`Jj2{@1@ zK@8YVi%t^t6xBZW`2U0S-Sq>S4x6EzE#u)xUr>S)DZk5W{uQjT-*s!W7X!r!U+fi} zv!(GndRYn8(2c(*9bC6U5wQfEXRfW{+7u$hEC^noVVg*}!6u>GmHpxzm$%o{DjySYZ5HQTb1b5(_wRxmIvGHUs^BT$i~+%LXtk54L`Q5I4@-3)HDj%{v+5 z_;{2{ox<&!J7|3Fz#|aR-ui~loeVom4O=B!u@7g!Sz4y}pIW7cNxCt&CaM4i%XTGW zYx{?B0qg2MR{{n|%%ISSbH-7LH0U?1)uIk_ba9HH@y$OJGhJsl^y*+q*e-yy!m*Uk^T6M+Uuyf+nRbt z0GNP3Hsu47gvXWoO3rZRIpBizmNf2A^o=EcxXFZx!t1?`^7_&jq^|=LdY|jl#GR*H zp1LZp5jNN)FwM@=D@274z^O7kuaNq4s!zdEYu5vh+gDA8!5`hfN_KrC#&<3293&E;}^>%kmBE?461EU9J zO^i}K7eCmO1#aeD-3jyk2=(e0JLa<~uGUce4ALP4SMR(+%mrK|ogFQJV9k+4 zaO4q`!a+5fM2$(Ci9;C;axtStRYP(d0T-z>SB31SA60_XVq>}t-vxZkP!NPUZrGcj z=w(i`7FYu}quvOb5dxU1v=M-*e#FNC_-ZP^2cgfU0Ur>X)nMEfpp4uxd~iG-tIngy z*lm3O)9pR-)0svO!NGf+N3XvDh0{oInQSv~nlNw^MAJWCeC4{A8|QW;FP$b_(28=# z18!3c7jT=}aj+*#@vng_QI+9Yj2Zyx)=y?E{{`vV00+vxcxfi5D(OYdJ)>vWv4Ng6 zVn6U9o^zkFdq$!%!gbgG(bZH1R=fg!^Xi}H+f%irLPrDAeuHhAIM93e4qdk}?&J z6G(EQDFc{AIAZw^Xjifj6U{>DFYM|;0I<&~`DU>L(6A7wzMg55G|b+z;;NDtbbCaH9*5mB~*=20R^+^IM!T7CEk#PSXe2VsT?O-X(7wRTYHYV~d+uKts}5CVgCzwB}Zu@$ec<3pVS z9E!;&7XnkZyvKB%7?^0d0=kE{zs{)<+N~pHjb9i~`X0DT5jTy)Ha+jD&?%>2!8E-T zvsl2(U4^1dD*wENWCWb3AblNk4nQiGyoZ+I_^UnfCg%Won>c3R(`zS7qLu`3f0Ts{ zKyjKU+SNE+ST|9credtp3GRpAF9R^kfHA2UhQ6=@@hQ2r@yE@SOp5}3=3^FizR2iK zGU4%C)vGuSe%_lS`Ky4b_#B$yv6QI|9=o&VZN!3nz#Gd)A|@o=F{(0>;rCk!X?ox+ zoU@;k|7N@&&VQjs%oy$UMUjYY$+QCh8qZUJbI-wSRl@Xt`-kV?sP@va%9p}Hj36cQLPmkfn=~$8(s743?DdB%hK++fAPk>i} zd0{5t9VT7!)D6F+VUxuLxK@eC!?iV2-@R6XMEXe4AqB1TGx9V_Pl>2gQ*d0Pi;jh-`LV>k(#*o0HhM z)6k7Us^_?@WjA|#{^$P!=vxHA>I-z?5S-^NIc+Zd9{SifqC%|04rF!oTyTlX? z|J;vocjJ}-@aDte)bo+;if)co028f@7k0C5rGak7=ji4_P8Nv-MSpn!^f9Rg#^ivo z#Jx2TL?RqX{gnT=6NBS{%8SI|H(Zjhz`sfaRf9)8T61m1wX>5&4Vwk4JB_;n)WWWp z2-zCBYh{?98md6+WernZ2YV$utN6(gIIWdGH83yk{h0bctK0!|J&2^wsA zELLFXy}`h1`inH~9K?P-52tcd{WpK+?~E2FdV!G3CH6Lo3n2S>0c4aHz_%;Go)Q3~ z&Dl0=-&81GV_qi6^Z@YsLvS7x@P?hKT8HgtHWP||Ckr8~ zpi0hvf$R?m3+|sZjaJax-5pQ9kBy@a#wxk-VB(LrGLV7spO3zwj9qv(7(9!!B{5~x zGV#;(%-(@?>0Vk;oEB&-2X7zLg=mNYm+l$%mdY@4E!BaeZvKtHQV5H(A1GwUbLclL z7o7vNL@UOLwMd~scmhPkSN59m`xfT__-}db|03_tqp57e_))y1P>GTTGF7HzNM>P6 zB1$r5R_0+7LTp1)X%J;5Qz7#_lc7||JkM0-VN-@J^SK`Oes{m`_jk_vp0&%Q;nzJ|~Bxx^-z7e+%3#_G477Qkg#hiF?SN3OV`VkKaI{BcFp@Ov>5 z0O%K3(UdQj&4nW}xbfl4RbM^?M{IR-k@|;$fYWMG)2aTgy?77#u&;K5KYg=${lG9b21opaB*7tL%q`XkG@b*UU zpTvQjq4hCgbcn?_*d4PW6>uW2&H!w-K=G~mq zZxWxoB<%V3_s{K|76Q^nQamK$N=}A8rzsEt@|EHG{N(7m2Yb92JjKH#FiwV;w7o(_ zv-2l}tZ#>f2$QQJ?t=bdB(kF>S2@@+JmtF9{N>Tf6!%;7i9+%1YIROikWfzS^8vf@ zLRML_o_NAWw#>cal~8~}CF>cTBvFSuEQ5~qVG4SKDJV@OIP(?ag0`~)AvI+jwDfbP^CLfP*sR7PkGNcox*N! z0}a>)`2nm3J6b|p=HWXj?QDAY`1=0nH{QLPg`}3v%=n%|B9*?UL|X@< z*TLWmM|9$=wPXRhIPEI?qi?Nk`5k7mt4r+SGC_alemSCeRM8U9Y@S8fqjWYSf2x^; zh4`@Fq8Wv4B3EE*=te8@6vtz_jXXYpPOk8!d{0mBE4;C~RA*3cMtTl8erEXN*WBIa zD*qdl%-qw?nx-Yd8fBy_S%>Ud&l8?+U^ef{3bsWxe zZVTJM^)*TSbp}m~=3nxfy4WaEwLTwOVV&FGl%y`6P+-xXbZhK)6_v0LgHzbhsZ8>A zS@1G=W~_IL>%Hp9A#zv$E-j5Y^18;@H|<(dNIiW=fAc*#27(iIHGk*o#@TFLoh~%A zwYl7|pDWFf?MB4W(p!7Omrz_w_8O9)4IN#u$Xx&!6MjH%ht3(Dp{v4H$+}g4`_a^>);Rr|A*T@&-zd*W{-dWyi~oG1f8|~jls)Wp ztU>0$QIzJr&wjGlPR{VKC**3^;G8fX!JiYqHEW@6!#zgT51pi*j*VL~C0T6(ob@eK z0CjYe&5iN$tQKoy+bMh053gBOQ%e zk?`{`pI^ZqO**TLW}^9F^DbJiYC&d=?h3uJu%SdWJ4{8F)thk6P{K zkHxl|&e%$%J|yxw+SYVi|BEghaEDW2;;BqI(|p!OYQ~$%LC76k;L;Ubb^HH+&BoZl zjjOuiVK=$-n)ma#HhEB6+$y+Z`~W&C2p2*}ghLSh)=kq_`N@4@FJGNu80bojcKx29 znR#L9oGMje;gCn)E3|}F=lP3wzmP9CWg;1aI6K)6)!u!r4PYPG=(~*4IzSMkMBia* zrJ%?3ZShEq$7*s9QkKKVOn2@Im4;d`Vj3cmZu-i5Wj$_#A5S;yBg-j`Urvb`XE@*Y zsQ#*BVi`LjT3UkP%k1SC(1UuE*Z}%86Y(XG$98>WH`9%Q0eKvm_VT@-K8&t4=K*Ox z2E~8!0_U&QKFXr@Is&?#_1^8B_~>36*;T(7a}G zo-$7l+U*`cYw`ec7z61IP^6fKdHudq4@%nxvQ8EF6ik9`qU1zZwfdMwi4O`Nge*$# zPyXT=5j^$mTRgycH6XzFXDk{sYbWDtPW}cGEG2f*f=>l$NIH;MSJ#(qT}Rt|@xY9p zZ!4Fz9LtUaF1j^6UT`D$2lm>4k=Eaj>R-&#OC8 z6_-}EB+5F5b#~tyZF-F2Mx0mmZM}siZMV`Is*rSF4a?9g)yR9JU7%W$EAM=CX}YUv zKv1paP|gt&T8KPql2=Tyh#|zuOMK(hW=uQhI6TDcMH1j9>{H1QT*w>i$t9o&XU~rSqi532xYoR z2-CFk;qG9QN^SDIEI1?wnA~3Y?F7SVVV`R5&JNFo>onY7-yOFB$Kit9DZ9rmKc5`8 zcS1=|1uBWv&SX^`5?$z9la~GiI(XnbR1lfM{cU6Gl;)38B}XD@Y}*Q|yb>~J#x~_E zrVbDaD#D?`C^4P%EtZy3xR&?M7Gb)(GfuS zWUfkzan8`Q2XZo-nbX90ZWbdtvH@^v4ZI$I4lF>`Z~r31_4)xlI5uC$>nxEz2VOi9 zCN8v0(D?lWGqe`11rgE)@N#bFyVa6=T?I}IKPfT9pE<3Mk}3gHWCf|h6w8I7{hB7?ZMGl(A$^$>&% z!-pPl1nYT9XYL|Z?hcTEJVDa&$0;5Ox;Z+q(2fz}d&NlHF!;N<7{m=-4XxD9?|sjH zBVxo=Wov|8s_wmIIs!P8ghC>E% zTA~n1@uCZQzFh(w^wpL188fMiis!ShG^Wsb7|)N?+UVNEY=Ju+8Tie!8QTxHV`*Kg zVknR4E5S~_K6GIeeaV-^pR`^gpch40jF60q?8izN+EgzkW!1ZDWsx0E0S)e#y2a!& za(Ds!0lEQ~Mc_dw9c)Kj9I4F|?iWf;mlk{vL~w1uCYX!flMQ7hR|6l<8sBwIo$j>6ADLdF0#kSWd5}f5(d9+W!?zi___hfbKlM22#6k89d9p(LQ4Ap$C>VygHYg|s)AS_nFwKmg}Tjk=4q_!{G^cSTZdLEFw zThdqwtk%ufcaa>!2uV4&B3%+s;-+R_5|iOVqIqdDOTqo|fT!)PBYlEpjKJ)WLHROc z3yl4@5T|dJ=ob<51X$b{xN444AE)GgPAn2)1!91_O)F5Nx@vlPI}m_vG1A*P0o2OM zKFLq^THlO1#v>t{3Ecg|7rPC;DIWnV16gJnPTRUbh?+Vr=leM?%3Wjy?Dq}5E4{Gk z3~7+pwb=xeAZ>T{Wp9X&uxC-(-e0ih>v8YAB&7!_dYhitEHaOvL|Y`Q?E51ySU9@j zcn_>YC1#UkGA6&!xpbvqJWMh1qcznZL)bx%3;8z-oVAMluy)6~A3A3Zx=!yEGNnHxgcy=lRh`u{M0ShbvTP%M;>!84Ti6NpZTRvFIcQmA+r>L#FGnpp=LXsoea1 zvfJmN6U`FBn}l^qJgmO@$Wr1k#HaY*b+K?!+g&+y@`` z;ee*#NLqfUwiiQZU4FGF*YDk4kHIFR_$0rho_CQTxdI0f?YTYdf$7fmsp@z48ewm@ zf1>jIE92af`Y@wRy*O>gomoqIYQngBzvHJta6=^{(|0{2seuTqi%3(2=xC7YqXDPB zhP|a5m&$teTWlD_edR>@1_Q7>Uo`a`)sI^rI^He3|L9+47Xf8=h!2#w&7gm@jC`sO zzo>`OK91Xe+B~PMO0O629OCThB-C$hN|L!kJB=>Ko`VN2|v`K{R^>%~A)mma4n#c7`Iwm6*H`ATWEMWgP$m*rMfriWX8 zm8EPTSwU#NqAf;!yncKS$u+nZ0$;pg-1hr+lOiFpI6CgX(O^+snovKNYOQh$QKk90^G~YFRkz^FP(zv$s$Ri|BOB2|+^_8R} z0oS>;E0sxkBfj)$`|QY8Sjy;BSvr(v`*vdq9ewZM{0w6OlJ-O3{@fL-Q>g=0)Jyj`Fm@SCKl;~A zLbSd~q}`enU`+frnpY@CPk**KEz$x>(@_~M8mV#dzS0{=*AWifdd-!dRKd_E(~H#C zU!ArTgR%iKD>;Q>+9>6kZ;*{px=H|2GTdM(wz8}ZoQBa*uE7f=kJ6KDKPOI zLFWcqoFw1G#jI%)=A$WJji@U%AT1GsU@% z)VvvkNTv_w!FxvKQ&Ee}h7$b`AR6L;cD;Xc?y&r602p3*nTI#1U-Lo&*C76@BxK_5 zl6`#owZl99sN10X@<2m!Vl-VTtN2l$Vkma6uh?ATHS#%V_sK*etoac8uSd0#-cYcL zq0e1`@~}yg5!VO5D!ukJ08h#&W7-4AmqJzBDq=*e(Yl)l4?Wdi*wf)JwONFD_8yn4 znW^6rFFNNh@{2rhO*E^)$IE*d2=e(Nj4I4?iEBd#V7;AyH`c}-e;I2X!q zxX4CM>vyR9-f<}F_?$1>LPja4saBed<6VYO*!qSek23zg^yb!g^%Q_TFH4yO||T~#_|$yo0X^MTbK zA%VCX67^W`ta=cd?M&Faj6%nm?`O?j5Cr~rJjwUJUxr%yR$m++hwRS{THp`|y_hr?=Kgjt&^=TzQjDGUi)A}}glK%qi`EW>&U993*{L+l_ z2_7sEcf>^INqY7zh(>M!+OL@GXa)d2MElfs-KOGcmn~%YaOQgEx(_A&*qBw+#3Euh zHxwG(ziLKS;1m8f^P^j=HieSZ#{KP>i5CNtfEj~0n{%w8H|}QO(DFLjEzZ~sAO3T5 z> {Pu)<=a3)WE;+Q`ccbc!>CKSc$H5I?K1##+$-pA`c%v06Z^Jy(h0WF!Ebqc2x zX9&ioeH~H&davB0MqNCKNga|j2@vS{D5k2Mh>~NN}p)lJb;3RJ}MN15LlL)dcLNxx)%#-Jn1Eqz< zkfgZ2DHAcDn_pH5<=2Q^D)RB^)7s|WObDw-_VEo<*c+6Zxn*_zRZ$*`aZJT55(1BX z7)vH>V>(4zOI~SZ(&@{{Jn!C>qhnxn!>U3+0+I(d>sAs13RKZkLlqS+pg9_#N^0g? zO{d!ypKL-`6^r1gj1DeWZ`Qy`xs-yXSjOW*@r8Zj-f`GKhCx@oAm!1H;@N zF5tT0(*#?e!bvnb_Zu?%vTicbLs-v(2)n+tgO*D_TSx_(yZV|n1pU1A^S_4GR%nGtWu)O^uQv8Ogt7F*w4Ig^hS7 zPui@`6I~HU-3qF_!Vr#${Myw!f?~-G?KlJXCH3Vq>Cxiv3fcw_o9v@vJaywchV%$T zP}z6r5s?_xGl<9g8Rq@c0VVz?Ni(q?*Ll!cI-OD9eGS(@(aqNL=+OX@cbk^@Kqem? z@!@8SP5S}yis}N_n^xhjN}==z3kpR`cNH^gj4~M}kQpW)k9bt~gbka>GXW^rp35&C z#yHdGr`r;5eiafuIsYyJiD*%6<5rXWQN>FYv%R-;;+dr4yv(rqB;EV9r+wPMMX5e^ z1)b9?ogY`ycFoIXZ)Jzh8elHDipmQz(FmPep6cEwVOr1OI%2g<+fiG)Nc10~@m4)WnkPrT8ldLh9H*p_w&n1J_Q%^ob4?;8_p|ad0H$)tknFHpxsW;?Paa>}r zPF(m5Msx(i>J|xj!xq_(a&=zE8Q~z8AcwAe1^5vt&%*DQd@wj_>S3OtO{9iA%Cl$gkQ!v*5{6xq6}yI zi)~I}ft`;vQ2=0r&chmW0DTf&Sb2ylLnJ)99W%u3y`T7+Q6!Cw?q{CaKZ;If&YcC{-dGK1 zAW-GvM~gw|Iq|51vm{I&f8T{e%3FQ804zqta80s|Hh%iwPadM;Kq=V-o%!T}$4$aN z8DB*qAhc9b;3FD92>d$g13vC#fwt+Q%um1=DIGYv591>Qu))&Ek>EX~D&tbV0s7Zl9Vj82?_b30&PubZE*tXi;;4;GY zQhobibDW6_kIIAJ2t0=e*hH0Q@Fxcmw!?l7<@AS+tbyO+JG@oEwL6OdizkCdzn@*oseItL!N;UmmZj)5Ld$ zKnivs$0zy`aiQB~K%7Jk36n#`0r9-e;+Zd-#2sp^0KfWhi2a{Um=pkmI42!INjOt5 zm{H`5a?b^bB1&fi5q0>9j1ouxKQ}J%+J}rQmh9;Op3D~6u^-^|){L48MA_nN1C8(~ zt?fA;0XUh`7umbSIUYmgb($>VE>X?`PvKqPL=Md$)qVIKXoP^$(}q6{{&To>VUpjd z_`d$Li{;@@ves#Yv>+lL6WF_44o2L>Pg;Y(8gYLT?i1d=i-Z!H_oqYTr39-04mmY) z$ba!=(GY)@jsWu1{RBqwSYDVo`1-M8&yQLBvscX%Vf;=cIby`&vmwC!`79L<;wY{n zLQIl-x>Y>2hRo&CBdwW@ML!coMj@u{yYe* zv9%@sJaP1tV270aq&h7AjHp$}ko}J`3ncz3eFruz!Ak5E!@&a&7EvnxqdAjGV5lj9 z0R&D`5cEFC$C&+KRv{%0VhmZ_OVuwVh}#J&TTqVw)nx6cADP$)-rfHRYZnO`q*W+* z=g>cY5(GKuUDW2k`rjEhzY9aZ;r6@R@t^M@(sHZzl_35ssSF0%OP{;~nZf_|GS!F5>!IB74a+ z&zKN91JR;%;EU)RVZ`0_5D1XU_YkE{9*>~>Wvzw&IX?n0g12MJ*>S|5X+fi&s3&pq zCyAp<;fBxsSG}>lvM37p+&|fI*vvWbE*)I6>b?KO;N=RZKf_0?kGPl3@56JiJRW!B z>mcAGUe}R*)c!M`k2nZ*#GU%D;%9q7Ae>6lG0O2~A5k(LNE*5KaNQa3&EXTFF=_`CbV1L7!XVcr=#HYZ_DN!MZhO>EMLOY|C1 z<2{n)L5B#xLB)Q8c?);%zT-?BKHX(_Q0CM=CxjAzXC#E`vk#HPPhNv3*;`I?|Fi9} z2w9L4N`rV7YvN%)UuV?VPaH*a5+cN@dGCDOG|@i$sYrkCBjeT!=AzN|dW|z%FUW zYu0K+I|3Ceu=~+eYkv|^M^M|sUmi5^#u1Gr@^E;rhVAhzQPPhu!vv4Eq1cHN3^i-^$B?0NtqQ*mrFR!KLII5PaFx{CR__C_zpK)-(IJ%R~eh_~eOwF8>@mOq` zsHn(GL4@6A&j_YBDx`92V`$4HYIvKB6g9f{uK}r=$Ds+lb#sdrgfdm%V!Jc$Bj;nK zm-_day^KHgSi3RiQl@5q$SdQQn9PF`iTA_{+|j(6?zTF()#mcXo=Y3^(G|a(Y_aQD z;7L_Tlqf@&$8~Q`J}mPq@W{Zuw5wM)XmaV}O^~qNsK@+8-`st1{A)pQPx+8bHrAxG z5Zl{zw=Z{T#wr81;nz2}jFCUQrEgs^V|8QkX2n*5*w|DIZon3EZEIQh-8e@I zK3*EJ=CF6(*CrA4UhuKz*wwgIC1vcX@J|67XndrH$)$onBCe1;8$F0`BHAlDb4X*UE z&l{@8az7V5j9Fr^cPjDB*qmGsbNMr%->`ghEAUBp=%u0K;`$qs;j{%GUOJUKH2H7g z?7N!QJyy{-E9Rso@>NY-tu2&o0|7we| zqjLy?j#C|CnEykeY!x#1YLQzVi*{XB;b||vi<{wY_gjt_lCa*e;kuQKn^CZw>z*&8 z>djWcPA;?0Ec`0;6w?g09~m+z_vGAMF7jK2u}`fJ?am-vR2?vQb^2Qb!{ss?3?40f z{deiSip-^keC(ZRGg#y3&V>y9hOX$4h5z5>_K-1*3QhT@Q;TaCpO|y*?rXcCIW?r% zycO+u{%e6&$8z?EzB5UkS|j)?+e-zGtYUB+zBv!M+I41k+57>S)zDqN1220g2Y3&p zZxtTTTX2y#z~<0kMKA|TpU35z<*aG(iXZprF%mBIFvYBjZbi2@%Bubh#R>9#SC`7I zu)w-Zz1_52-t5{6=v`=U67#+fS%H^A66SK7X zh94_txUI#;*GE^%N5W#bY*Txi(7bXR^|%e(&BQYkbA=QS57f``8c=U-=|7r{nNQfj zTu=?dcqptcs~CDHSZ1QO7Om75D?&*&xRT3ut;#8*saFo*H-Q48FAw_>AeYLbyXRh5 zQu?l+8dMK`@_h|`vyZ%_K_NfU=#TiU!qSC_&94^qVj)HKFNa+G+QNnx_0A@B_WbWl z4dp%6nXaQbv)RAxsxa$3G;=7RFvDy0%8Fgm`JP+~c#j(;Pf9Lxng8(pJutZ6xgv{RFBiv2v2dLH73g4f_>)*#$ z9`c<%OCv3eeAItelPT3c+y7mC!hA4!@!&mA(2D;T4%KUFzl>=^Oq5WbA6hKleQev% zL3x0OEi_(nMiVa#W@v-dVHa`Z8@{0VGU=m(G6Iq1m|ls1lPVZa%qjb)E8V$mF&`6M z{1F-aI7~j@Gh-1_h?mK0YS504uM1S|Tnq3%>BlS{fd9WpnI>-2A${8-inGcLriMEE z% z4J;FqdW3_vA685A0ZEYrez}hFDqEfi;EiuQ7_|_bD`6s=AN&Ja3{n48k8L+w$mIS_ z@Nfdo!@W4tV>Yuhkt=GkL40R5|E6&z3C1^GvJcKTAx};x-R>t;g5qo%TRirc5RJJ8 zn@MSY=AzTi?RWZ|vcidglaj~xM_#Pc{-Eo)D0gzjPL}HnG*UXN6&qyc5KwkP96!Ez z7;hrx_YRW_gqoR^txp7SJ@9a;$@_SwL0!^7-hFW3c$R^q{xu%(w*yFDxC^n@KFucV z@&8GWAHoOlDD4UsJ_OK3%Sy`^rc3yrdWjz)ph!-hM4Y2z?Hzj7kzKw;2#i zhUjKKZW8i<0h4WsZ!wG5K22zdl4@DT^r~7ew%g3X?P{YuVPBEM0y7`AKn@imbW63^ zZoZL9F7bq=%w|hV?g`42g(tmqDpt4A{wckAqYrKzM~H4mpoTM4R22FKC_jFWka>(i zdL#r6a~+i2w299Q2RdH|xMqM8wl|@pwP0;|&bHBUy7LAkOSAH(tvb`*00GdobHM>n z+^m~XvFYlt0g%XG0CnlO;$}Djf_^=fX2b`oldjl&{ByM}zOi9*ZH~!!A&=I`>uyZL zg%5`2gv0)5m$V~=xfpRNwpJ1vk*?y-?K@g-!tE%hM2+9?~>RLFr(zcZFYx`@Y zYd`mhDXRuNAglIGDm~G3uH>a6DEj*Pv>)g#qeV3tJP)gB`dIS!o*(OpJb3u(bNo2} z2fLa43F^Wfc~0s480jNwYq!qnj&&RP2Lr2#V!C#Iz0h&(RVa&W>fIlAkL!~Gw{TKt zx;UM%jkSi<93(aQF}O^fFXpz6O=fc;r(@japIetLbs9M8T84UY|Tcwt}*O_48U%Gyq{VrmcGzrDuum8Q0D=x zaL&_KOz|M5FPCG(_#VfKMXGK>6Ql1y70E(3HMM^jDfuv5hMK%!D7RovZHsZog?|K6 zsIm2*K1?>Bu>fRMLQ45@0g^!d7!UbjizS8RI7;Jv9SEA?`dgufy!6y^!5<5ZgVX5V zyu;X+l!vF=lZ^@{p(w;F`Tg$v4ZF+q&C~lW1+#!vVul5&*cY&Dmp{glz&zj`!z7CGo199I^c7^B-vS_JgB&&!*T=}@ z_S@?tEO_fj7YET_&(6$PEUZF>vFb4%T@FZ#g#zlF6HCYBakx24Yx#bm&rp_F?jSUX z_~lWE)*1bHY{8mmT(TV>5(WJ(BGvkNhu{S^$`yr5iZ!=o=dr|G6*67BXKUlUMor6l zh04OPxyIl6VnilFhAO5+v=z4;O9&yrP@M8AG?{#@G*HR-R_f4=i!w6B9cW?z$` z$hLkX4J-1__B+L_3V1jU(;+HyT;fuO=DkiH*IV*NNz0RwI^sx&%QJZx^WE(04r}%R zNQ@8zrB{rZMey|Fi<#L*)e#GjdAmBJjU6FJB#km;$A!D3k zljx^dObnEo#79b`4>WZ;1J3mZlI7LzmuR1a;+>jv(`hqGqMx!cgA`>0Vyjoj5qP~r zW*=t43eXL$V~rF)2J9irbzfT?!4MaZJ^sVVkSh^cR;1df-#-?C^FCAfkmh=6fzzQy z-FI<8Vo!SO9Mi+h95PlT3EqM6fr+j90D{(~einqvzVl32xpP_iR7Z;`^~MVYdhxlB zxfHzuCpwH;05jH~`dS}T#O0P3#KM@V)U{1=8_*tk$8&Z-K-9bi& zr8;^w9dAToM`ADcJ1u^ES~EBd7?@J{$x>7=6d$WLzL*eDOdn4xa7qq`%L%X#vl((bM zf-0$+nL?p(s%aheenueQ^!r_9G|l>`mInuEn;SzIW?l3otf|aOY4qJUIS5<;ZGQW{ zBCI%lW6+_>`h_o@;CG2}%6m(9Gdx_pEh)c8p;0sGDvg8PHac&ubDm=cdw9;~!_JSe zY7Ajch6}i~E0SAIrR00hzL>$-65FA2_{=Lb6=jyc_^ilsFM_G!O^8gFt9LOIKe`;5 z`{bpL#@z7w?RX3paOBRJqJ1Sr<1T?Zl{atesj}qbvfu)ry=6x*eFM&c0&EXA>&F{9 zx8eRi>#*i;uc>a9@qsOgXSVJ>l%$UwJcC)98jbFv?!r8f8Q0}PJDTKEYC3WqmH>$? z+3)=fg#$N4G3Vc6X5fO?u0KWhy?1R&3X-65#Z9Z-bQRkp(WY_YbqV%n%nSpKL zIsmcc9F+7#)+vtePqpHY?;Jd~W{nfRtU%Z=DsaCB^0)cYY+tZmCn$oBzLam=*T zPz_y{ZOzKDX#2qC+s@pDxgO5{HL{CZwd(rZPs%7qLHjW~=pb0^$bLP{kS7$hPw(&U z*<0v2$6kxX;^n^+asKrdd*rvC8_h-{ zs`out^fi4OIo|U8g2cp*wbd~17BEw!sQm{BHT?|ACoDkN>e-Dk}&7#}!P71Gd= zsq)Bj?!~00fqRG0TzM~TQ!f?08fMQpRH5{8PrDCquvEJ&jD;(f92wXY8G&;7z?{_Y zg0Y>yHvxNkakxXsk)F0JakCgoK|bDC81zA981}kOrRrzS#)0MSxcnT4_tu!jz~5*I z=W->$F}7$p1P`_-i#qfjlDgxN@w)G(1lxI?A5boIze?WrS;+JT)a*VsiLW`o;$G%Wc}S?y-InV)X$ zv}M>$dY4yOQEp;gDvA(dd5AJ5jl8E6i9V??HP>x1JTg1EJd)OzeS>VW*;I46cV@-5 zE^Q@uplMK8;*h9}tJ7-W^3l0e>U$dX3aJ|D#z0}E9<)B1lRKUxqN5r{g;2(<1EZ1f zTvV)fr1~cdw1{+;h?Ko~DX%UX*C zIyHtE`H5bIvi7r$DVBBx->A)~taR$#^U+vfL7WG%MJyoIW+Gfk1N`W|f?(kOxDtt$ z`qD#E73u78UZnttK?;4#v(b~zLpa5{9?t`folN|8zrN&&S$;U=Dx%{lUl+n{0*K$% zI5{eZSpahd+YFS1FXJ)^-oiqQ)-CR}xr&NE&1=&?t?O8nl57SsTlTow5K#LDk?z?h zooQOLtm49r{w5JtN)z1RZyYu2D4GoP0L-*@-j)(7MtunVOt_pha*L1I(wd4rJ_K|} zkAKIic4Ka_w)oXyy!Sy(0sfQxb($zbv^5h|d`JFWRxc*TZH^*&q4ubx?H)yLINW@8 z1(h{gKas9MBB^h__V8a8=pTf-wARaJl<`sMv#nENYkH;9zu}rW-$=C;tR_d_Kug~D z*5V%KhZk9||CoA}p_yV}o&(N1S1#`o5WC^xM5NL<5HoL0YiJ!PPmY;wSuRKJYo|k* zF&3|?D=14~aMpIXN}&dEuD7EAB9<@h06J0*q&J|bnYZOetXI*XEkY=j*{DBkBQZ{P z{;QUWjpX!$tNx5w{kNhHA7+3FXV%O;ZtE!=N+aMF41yMmZI8EBnJz@q4*arCznk$# z(}D!b#g3VcmbM$qr%P}z_vrSTCliADc925wtGmvBXve`ka+dnaiT8K%8CNtk-@ZKQ zIr(CmF{+W;eUw8VC8IqqNB*^k?lWm~qyPRfYvblvX||fqm+;A45vicCV`5hBdT&y> zGBh)EIqANO_F04#+z(!WJLb+HMpJDqefw(vAU)2z!F0w3KznSjbH@eRB{u&KW2^JJ zpeJg4`X;c&WcC9kzJ*C7H*dY=T}*4wbWhs(f*7n-hDY!bH7iWxVDmgM-$Z=q(A*M{ zy3YxvREZY~m+u$sJ3KCTq(M#(JF^}e?@|GjU*UYNU_R2C+1jl@NuaTY-1L;aUPJ27 zT#5EH*_X%W5qDEHoGT-0BiZCfrB^`_MWY~U{8==Lx@m0URRkfghEfLQC4H@x03~P- zL%qK9ET%HzVaa;3qeotoHOo8hQSGnOWmRC^umeSIps0z=4KN1#Y({z|}49&Og>pEip9p2iOsw z`8D;VilyG5EE1gP90seIFV!9W4p=QJ-s{9ow8Tp9(aJM_)1GNCq(3HjnM7Lh?HK{< z5`u2t4OtK}(rXt9F1!Ot$H}M0v)3A?CJo2(5tg(3i~=X9h`a{a^8-R@yO*Hi2mO7* zqZ%Q|SWa^CIH)YSv8d_-C&RcQLeSLHh#Nt;Kc5Y>rm|JW7W#D}XGdtsu`yEARLa8L z6CrDqh*@&akXcK-gPR|%YfzYfa^QGGQ@DwsZsD5hw8lv%t3$TMx*O*NlRBLoo2&<7 zw4(&icD*jT86%kko?1~~4tRswH#PO9JGq=xIw3akDCu|6kQw8j~C--s>)HogXKR@RbN zJ{d1?m~K~)Qdy5dudT%QubL1`bsG7_a zRY^<{S}NJ>9tOOB0$j*!Ke*ZYm)ayvsGp_mY8p#wJhicdRbf5_FZvlq=1TAcWyz_Z z9`&LPe4p^RF`>P-I-yT@V#;D5>It_7RcKhCw0y_qk7qUgKUXvkgQB{)1EP|{J?uY=lNG`qI z(78o`5afWPl6ymhfSO@B&U}z!g|r2j;pO8&cvqCfz45~1gUh0l!-8;^TPnb`Kg1U! z%^`mnJT%MwC)!;AQzOR0nMdIBEHTU_kWYPn^hZx*jfGkB&d^BNAl!6^1zZtAWU7qtMTTc$4w5F2ue^Xs#?nJo^nyiAEHAkZ#9#u7vg3 z6Bo~de=vy?wo637m?V7iq37=ChRt*iLb#|0Hbf6wBq99v`T+IQ$8L`%AUASW1@_?S z#3<>V)uIlADH#R{5aO(Ytf&(zs;&J<#_2b2Wo}OeEi1v8(*a|)?=e9?P}{@jU*Du2 zkl6)pAZ&q=yo=^LLnEPKPCDcp@D#tVV?V$_vQ&szLlOa-u%iEj`i&tr`%fa^G7sv+ zk$H;u7)bfc!<3#!I_=wmp`Z+5N?x~Gsqu1Z588D{{>C*21hxiAi6+x6jUfDW?_~`u zYE9Ef$V1iag&c*X^nI$`q(?X_b4A~@HOB7DiTpZDiBEwJA0GFLI0D%O&dG_Pap=+$ zs;Te7YQ+o@(qN#Kx&{UFCO|>*8d~a@1yf8vPf!f~{^dE%jUl+WyR)1EJy?}NvLx(L zEO^Prf@B|t2@tMa!80J)i;Z?gbV96y*@eMNWI)w4hq8kll`6IztEbi2CCRh zAQ~5#no*(vp(L7+j28dL$O-#<{NYdh3HtzV4u7bI2ljYHz}`-FSx6de+1G)@J`wV~ zSw4t0vI5C)nHG8jnv>Lm1HlW9T;MTo?IM%UtEQk2=izUPz)sp=`8x{-QR-dgJU{hyk+0O1HOHloA`i6z#e0OrOUZkMb&Kd^n zp?e<_tF5W(4>$|wWRSa z-TS~BbqJ1?sImxQS<69Bx~twI;2b`~Djx|6C-;Pxq?|5g zpX3Qtl~g@Q=;8YmVs?^NmyoPwdTyk4XLaxE_}ZQfO9}?yGW>$=6}|8=!DjX0lENjZ zxnwI5>t!sNaz#iJhrwyH%&;g%Y=0Q$6xW*+> z9c7V@R=x-wHC7b1snld|eySEl`V;Fc-S3@DZ~_CNVL)A3sC(RTFO7hFnT{a*}48(Ht&qOo3mVi=`X(Ylb<3egFZ`qy?$l!apWM$=a38>9|aS3LCTqkw1$Mr@w!`=(W zmjMS3Vl`UqY3IIpf7+LWtfOS37K;324b~PX&j6>JWABOM=WjC$4?c(!t@h{q(DIuF zlI>%c_#z1qXaf?`j`A=Lf^l#Mu1AVC>&KU%rrTBlN`SQSI$fmeMLC{FK_2|W01 zs8#M(e9Ec&PHVA)S+aU+NU;zQgXI=FoC|(&Upd{|1xQSdfk3GPl?sWYygc%EmFuqNiP%xWE^j?l8Ybr7151Mz2izL1cIOz zv@nuffmCRDntqwsyz~+%;8Ccy4hCPa7F4vMhI*z}?$xn|x9pJTxMR zz%uxLZYjy2qMRZTo6ljpacAGidO}|$lB$$MDqxiKM9Y!R7iEQlUpj~3NWTF;E^K-I z2}%-F;fk@yHf{jyCJ(rNHaPxEv0Vl`?MTYxGj}tApiRiaCP3;Iwpn1epQh$BtJ_0q zS{IXb3Nw0eGM>sRg?oyBz2LelZnN)LBrNcc^A`B7CT9Uv$8Oho65pxnFc=cg|B|+D zG?PzQ&O>WF`GO&jwKT~drU>i4g7n4N0r6=ucIq0$*5&++;1UpMEXP&YoF}-y6Vl}z z2qfeJ<^9w9NJ{dbZ;C?huAb@|Xy5jk8xoLN%(X4d$mx8oU1fQQH08*T9^GJS0svdqh$f${Pw*us0<){~EPnHl)AOa%qH>rC4=ka7T$d3wfRv z5h&5!4KgD_^uEZ0b$2zeR(WTf@-JY?R7@l6Iv+o@+4=(t8Oe)}f~6 zv9$oh;l_P0%$F-*3|(gnxPu3;uz*Rq|C56bVa|HsB-|IbMCRPVTA5p1Vr;R4F!eG9 zgd@9A^hwqoWtR8pu8BQ1Nqwxa5AuoaPO**wBR&0tQ zeBQCP74M7H0(wyC+og*NJDwG2rT{X<*71!sVahyF4KIY%bC!G+tB0-Um(J zicLcN0<}>1{8T^!p3x?N1#%R=E~Y^{pZ@}G&smMOT0`(V*kJ1$@xM8^lL_$ z@D|XLezjNl5AKNN+*y>e9?Jupm-w7{0(d<^HgXNEdlxanpoYTXZn6gQ{?>AL3}V12 z*uIV+|8gB9UC$Qu6Ti?^uqc8D8?pFnCDsi-H&1EtPQ(j>JemjtwpOVVTKg;HDr-R3 zo`vA5k~lDB2XKIA@jSONY-iNuN+1alMQws-L5Fx2Eao;wq)^bz10ld5&^L8Jx5bWR z6y4VURP&oRGv}7LLYEbL$Zx!Rg8GH{zMEXT@Ut<2RS5kmXM*2>NsmFctnGoACPoR@ z8sC!_M?edN0-r$FljgZQiAqt%z;yox+Jx&{cBG8z4^xs{LyqkiEwlQawIT(uEJGwK zH{q;NA}%VmK;awpRE=85b{lG_RzQka>&~KGH3K`e%ki5?nFgmqof8K}bPe=|7=eyL zK{teYu0VuExW{vAW1({mQk%v|1{`78+Q>qhShR)e(t^{mZT{9rOXz6lnX`a_5u>#c zm3n@hItj5yJKiba@!x0VKyIQA4f6OM^a^Ha{egdR57Mk=B-Z;-0Hjg%7@-C_Z4)=$ zbrUg#!=V@E4DcLjb(OK(6ptP|DQa)rDcMD*hICm~sqk>OwJQeN>+7Y1fj8a?1tV=> zBwd>Pq4+v4N+F@Yt>p6>a(e@=!EEo04a*=ibu65eT^B6yyv?MDmb}+`l7Ht!-$JzH z>cOjtTTs6k1PS|Ven69QQqWROqgs%(lS#Wh_AfA_)t1=Y{0QvL_F$3LK-&Q}2nyv> zl;@)7L>1R!;Kv}>1M-Naze zs<(d$?emP!h_f^bEgJ)nMqLsG=)RAq|CbJoSrVC%r~Drz3(pmDQ5*xEQ){F)|4`9L zLK-PKJanK581BkoA6?}snK%s%#2xw$&`fKP4a`&3!kp0ZeP{Y831Km_>?cv#Pz?XB%iwSN$>4I zX#RA8FCz!oEFx&TUx8;OJ`3C;%KHwUU@miAIzsuOtQw#gwn)>Sk=HVZ-XI`@j&2AC zrbXWSOkod>qRY#O<&uP&09M5-g&Zxx-KU_c0nvW#33zDzFJ9RFTcW0q@!C%bw4akI zUGMG&fVBW8-kpLUMZn*ayK14SHvM7}vQnbcB@)@zJ^YY8_D6aiwr1Ut0ms@s9^_&V z2Du_g%lg7X3QPXe4v0AF_3pPQD?;pm;a9@(kW`t69$a-0dgcWP7Jq80W3V6@NiZBq z-D~c%_z=e;l3bVa@kMS`h|Yb16M5F=JD1Q8NW*r`NVr2_^*ivM2X;`E()!qJ*#T*l zsorvQ^J^{72S^9fVCd>}O(X609SzZG%L!fm6cHCS3_`AC4`{*ckrBpuwyP5Q)wX=nZqAU0Hu@i>X6;()4vgU?6v zo;oQrh&vq}C>aKZH2(>zQ@t{p^583M=0FS}m%(Q_g4@VI&YzK6YHK|gNx%r{cy@26 zx3r$8rzV3zgUH znilqXO(BMrUQXW5U5J8Wt(3qEO};zPQLo#uv%Xl0$w-9(ldiU(WNtG&{k_PfwZZZKQE!BMlG`E<`ujJ+1#6dv6|21>5$G zB86nAsLV7GAt^EsC6zJ{nN!I;ghl3*p;2z3WF`?A!ZHsfrOdMoidHGl zvX2mT5zv103x6*;zd_Z*yyY~p#Cks$SU&eH9%yj`F}_j(bnI2g>%5$O0062X;XOven`QZNr9Nlp3FuMQDeqwt4emKQ+Z5?afq0W55TvfgI zdq=@mx+<2o1D>b-0$_D~-+gWY|6C?#g01AoeY=b31^7iT<_vh%*e9^fpp^8w9X+?)}!m`jhG#hc#Elg7Nzutj`0|K28KB^k2cw5C$z~ zI*AnY=pMLaxbhtQB4mp}egBXudxVEO`&`}*m&`$r34)+!3_y8Qa0gA|m!xWfya^SO zJYod*JDlhP(F9~0tKyLLtN)w;p}3a^Ct5u8tPyXf(JLYASJ{eZ+Z>pC2<&+sY$-Yg z?_qK>HMsJui5>9>KMgDPRe=NcaFl;Sv}Cu#S9x#e8-%?3{_&wHib(*1rV6Dl5hxg} zNQPGb-g4?aM}vSbLrw(eq|srX@OV`_ftBnZdKmh?MLeO@+gBTg&e1qsNf??l280J< zFMAKf#?1n`<9>}K<ZTVWM!pWuZR+>V%+X3FeDoAt>_dEsKNjcaM)*Jr_5v*=8 z*pR9Y$3HHm8i?4LQz?m5-C%q=Kr;L@5aDto?lb@_NE`<$P)kE-#u+8D)#F|`ob2;U zJRk7Mj#z&<1*?6LT|~Qf3FKvfrRmOR#2P%$Qb+@AS5>aS&wB`x>-erA0<(1Ek{91Ae{(rhO?0AxG`!M7grG03oX-r z+opgUYcx`JH-sPvfSD8kym|l)^P6iba4eU|55>>s!S{PT0|&7UWbX)`58d} zgn>I?77C?b|C#4nC-`%$1it4%(!+AAWp|83cVWKmKC3v zR)?M6SYI1}HUUC0bBb&luS1q@Rw8;)=q&l{`OJIg9YMw<&=hXhoUaQBJ|bqgIAH zQ9S(xV=D@{?_d+x3==T4^vY~x(H!&Yh6XM{0JVHB>jB~zw%6E0?L@H=z=Bbv2Cl^J zJkK6Z)i)qfS3mL)b(0Y(#8kIU5f`}&1nQsXuRde4LQv!%8m(yn*BeOzM3ZqsR`#gh zO*M+Tpk&G0j~w53n=eGH157`tgES zRO|b-kjx!u%w7%muFs1WOyqXeICBCW&H2>y)I7|F70$gYJ=qH)T$etlndEP4?m4|s zBaIqIopMUuMN58x5hNjnb&jABWZ@DnvEwiIZP|X9#C}oJe}?S=u}(Vvh|K$c&5!*m{?LUy_IO8{dxof~bvv0(+^J-!%j=;z?I zs-|*3fEZ)-e%|!k8R(sCXzEE^PoN*@C4v-i+H*IH2o90g9)} z-Q9M0yv;_+iEa!b=e_^{NUf&BDj#f|{#QqLu?cV|=Wa2`vyB2ST1CXW8GgbdxFcd2 zoG#~upt`-9V@7RoF`u2V?Y<0sNbcv@cTL`%ph=;iW4G2e0!GYQWO%!~&%EGEE**}4 zc;@aj(e&1irlTZ>h?N-6`A)pXelPE#Yjz#Bc<5|biHUd? znZ%vU^;~I6B%H^e=PJ$(Hzo`~oIgmoUQz=Hz_W6G0mmW_$nhrH<_n|^$%%rrP&;||ykrI?10Q%RmxZbbw$^Q(+p(Rge~6my zq4>~S@rh{ra zXGQIL43s5*p2>cK-PWKt1ceFD(DMW4BefVX()YvFK)LCUtIxkzh4LwQN2)C(=2k*4 zYNVNOa99~O;y{yS>&$~3^DS?=kBEzxtgBTbPpJu%tvWz|>A-@#bdn8t{!y$Z3!K_U zt#%4l``i3=d}In>d;V3n_YII@e--t`BLn=g>jOPObP$h}!j7Vb77uKDRohUsP2DS^zoLttYN1jd?W!jyV(})Zx>YGdv;Z@hS?GPk1~t4_F#~4gm7~f*O&} z7qrFd*LPhT|N1uG2Wz_!7vNuyOp{4mQN-F?#4f1cQaoyDQ&MKD6QBBatUbey%o#^p zVXG3X?L$Y&zy;?)dGrA>Oc@52(;g~fIXdS!sBM#&xc>Pm@H**nF@^9hPSWuVHeH6- zx})tHFb$uhj>f*(_LW&L&laZ#u5v`GTc;RJqztq{pE>=gFh%dKEzLV{yj)yC$2V7X z(z$t;C`O3bY@crq8)77tfxG=zi2fG@?1rB9N#d<&N-&y^9YjWxrK)V+)@B$f~Cqj)M09kSALpM#fh0? zaikz<9?Huww=1eI9=?1B)f-1Z?F-4g8)mz0%jAH=wBPq{XQgaGsF@cngGTrfDv)*Q zkvtA%C*t&03dD66R|yw!OU zidZBIiz<8v7>a(t!v|~fYR#*@5Vb@3#iK=R%$qBh<$MDVr%4pkG7(Dw8|3Q4IiJ!4 ze$z|PaR208At6{B$Rqo=6e1d_ieVA1%k`h%KOVxC)A58=oIUhs3w$Uc?yP9vG)#y) z!$`6@Hs2nD<;>{H^GA=@$L z=Y5;I9)hxSyI2qr>+fHKfAE06Sa?Fza`QtK{~DD-AC*T?$}FlliOmJZ!8y==Jkdb4 zxxkkw>uUOniF+e`(}A1FagJeg6Oc`NXEgSp1(Ci3cJE&q1AeO?H~3J4%1*Y;TL)|@ za+s|R1%wnbEL_$B=J`}&;Sv6+{CyDU4fkRpvTghxDj)vz?d6G0jz0P=F!;WU`%iCf z`n^E-kkjCU=*?Tx5($5*)pg|lv!A5ozW>In_pgXjG=bPo5RTNxVg8$vz8?zw+4{aD zjBTD(<8F}jp2)^)f5!G1-3tLluWOS?Clb`pZD7oqP@g6i5Tm0&=5*q}uw%7C&c<-< zQ#aAoh3#}BgZiQfVK-tqHwYt&tx>A58xmq`Fw*NqqRF2rAOebifd`{`fG38Q zMU+lUgt^)Fw&1TTa~&3ZKv>J^FD5Knn83~F3ZFLDLZ1Vh`6p5Edx^57Fafet2AdDe zxi|2kt4Yy51m^)hrU|uTg}X+(|GH}9Ft|w!*YYM`6J)>djN-H3#JN%;lmq9v7k|m! zyC*=>!EvZ+Q@0;Pkr@l&x?7uP#S7ZkP|eGBlZuO89R?3l-tiYT=e;DDr{qwm+U6Av zfxNL{GzWHbBzm3;Fm)b*?VF1^cm{rtYj^Fj%}L!}qu%=nIJuaFC7d zut3#|TcZ9PGChDo2EIPAY3Ey0>}El*r|TO77yk1ZU3`*BxFA5 zty8{mcXMzdm{j2VgUA0Ok5Y6ghy#r69H6X>PGeg~^H10Xw553;z;EDLAX)TBg70IKALcqoRYfo71Gshc62J3mvCD z+-$?`J%3F>bqzk$YFERyxt;L{{7@p6@E1ehPDpbEPWLww!KNT)mY}=RMf8|Aj}LLo z{0)Lf*B%TGwoQSO{U3t^-O&FrIMA*9|FyvZHg-x=A!e-q`!R=274!ef;te`9?Y*(! ztviHj10cucO1i$6G&E+58~)jp)r32nAnrIz=yps_ZLYeeww6u3-N4oTef8u=uAzK8 z>e8QG(IUH?B0?=ycD?quAt5=!eEy7_X61L6?$FU|4QmgZpJR^R4BuNe!9w6uG1%9=5lW@ zcI(Ri_zP!ZZaj41tha+_>3)@<=ngN$h*v+J)H~nuQNp%M8Bd?ro12#5hVxC$z2egz zyOK2X(kEHjxb%Z_WyZBZ$*$#a;N7dY+sIkFI{0y(+j!%GO?|GFamfS)PNrEAQZt_G z!)1Fj%A$5pr5Xti^>WBzb)GH|v>aq3LY& zhGmnT8YpgYkQUFV+enPusGf8E+@aS)JKVlJseY!pN09GK8r+>XfP6h~8LOTTBd4E*f-nTltvx%t&TtWh75(t#sSk+Ik`|^8M&F zULUFz+=xz{Pfee4vDh%Wu(AO3X}DWNq@g;J0<{EURi~!rJ6MR0=qQFjuO3R2^rEgT2dY}lHZ8XgP>{@B4-IS4+aoo+c$_&bjGE<1ec zx0wt700LH3!$5Wn!FGz>4?$N(lc852%d&D1;NR++PjpVSkgY;mLBumsq8#$#pf5YJ zG$kXJXbBIvtQc(Vz_1$87q@Ngd)6*eR;+zJwi`JQ_kw_kKjOd{Ssmg=?Eur} z_;fO{7B_S!x4?*hk zmKhhHoURK3;;VkCj|la_`IBrwI$}2+CE=K0n!JcQw~+gjCFdhawE}a{`WY{yX_&&{{7ouJEjGuTYfyr zg^i*@G@?GnTQ7KuY;=-M5$re4FCY@L)5P^f@AV=E~W1F2)J}FLiTO_VE*Hn zpG2Adx7Z8sZ$`QTQA~P6hs2|rSaBU!J0{3EvqR3A^EDYz@H`!@$R~&jCqjhS?yV!( z__Fu5KA<^W+~uW-eW4dHK@;cb*P?e=coo>~PDmLx2?46OQAi7e0(YMtv*>8Oy=)W% zwKR|R3GKJB9(@bp&}smk@gwkLj(OvLfV%M>I0k)qm}I<)0Y;?*$#Y&r2@&Poc^(1R zgpET-7+un+5aHX&M()Yq;s{r^5ittX88i}$sTX4qL)qIduVH>`#P~KJwq*RQMteW= zoo5bC<6qTo)JGedwgSXb1*+CGObZOQqIw8d=&oFo2qAZ!fE0kyT0~wBz#L${*o~@zS+HCUxKqw}7f z+LSIpF7QsQ12Yk1<`G(bcc(X`V2|7NNa{jT_(&O${XN*02zfmVJx}SE)X>8DgJIML z6opj*geAf#_Ol9#*&^5p;!m@*efx2b1As+eka$1=_%A?;7VukM_9&;~&X2JgeJm|H zTa-|VQr4)=v8!>jH{)`2yOZ&qQe&a|!Pu!w#26_3e*uN!m#?~V-Yl0zQ?=yEdotA3 znJlLH)YewuYYT-cqt=s%@9L%d!sM@VYQg7@Wh=O;e!!yK%mBVA&f(Z);3)`#L~&L3 zFu+fqp+>kiq4UsBt{*TUDDHe7g53R(vkRgT`N~;zt@YMnTR98@xI-P01ISC&xA%8F zt1UX1gl~L?3R(kADH&0o!H)4HDPjQRzoIo6cKcu7Rs~?=aCgo2>#Aw`w>*|dl5Pk< zLUt<0Ml$TE?*78%%XB!}(>hkr!FTKz^n9#FtWb4U=U%58Vi+d?7-OnkREp^8I^`(X z4q`1^cNuP70lb0|fNc^X`ak$$eGZb~4<`4fWgrv~G$&Iyv9GPQr-0VbB6R@(O?d-Q z`g?Mr<+crA^>02zkld`RAGa5LyF-nVyrOc0kkqZ61MqRjn_~;<$#pwYu(9dMZTy#@ z0piAUH(+?maT;m*SiG>FsGnrcWQa-o(X!3~HKrDEDS5@E?a}}w*$4)z3`Ok>BYvQH zyvvl`u24M^MhoRMdvS#a9Y1EK>v;hbCg(Y;{X5dhDkI1EM`cbY3v7SHDGt>|B|>r` z-u+A&sEHdM2o}NWIomkN<|HuN`^0PSYzaHC58c)Mn67V#}Ft2IVX?vTe=50`B78 zykZIT8GX@K>OUfG8)dg2Y`*oceq#sPU%MI`%^=ZRrw`U5@;6a>?llU|6_4Em(H zdAwr`csU+Z_DHHMxo~N_l|=3(PyQijE`7Rc$y>0NgZpA?))DR#m&CYIk4%x*9^rn8 z89VhptE&wMo zP8&nN_4bhLtpQyWE1#&cHI(TiwJNC^G!QbY%Z|IHhWA?621wgQM`ioru$yFO-5=)Y zm@#7Y2SYTJ&lkPo_!P(%VF_i1`k#6dAoG1BEn>_*SMR(H)EwwyA{&m+0SN7t)=e@4 z4O^gGx+sysfl4Xz9s>&ax}o5lZ?<}mO?yX7SB)RrGf&K&(H2n9wfrp~qw76dr+lO$ zZG;>GyQtzd>YjLaULBl+d*IRqY-AhHooiS6pkU;pm`UD}BZB4&CPF!VXp`&baScGP zY3Q}h_i8(on`^XD-O*5Ph<2E_Q%M~oZ#ZJ+S8`<1x7VV*c)>pR!P`qObk<6TMgzPk zW+jHDA~(MM$Q`pb3mzEo?S2hxn{8>|hukOkq}sek?Ih{!QlyE2 z7VR}5*~@J<8W|FNYM0!XpJih~EPhHZ6lQzhtkO-J*f!p|KHc}>lB-!4ZF9^v8s_J} zz*di$d$O(HZILP^sX_qit4%;bqf=rk$BuR#V->t~5Dw`A`%3IMsi|nN?f>3vHa^1G2BmU_qlr zjMb(54DqjDu4MVu3k5u{-mK3plx^T_#-LKwxqij-$D8dc(C7ZfE1HS2>`t+P@z!YP zvKssDZLQ8bmN{r6Lq~1P=89+T#vL)kx>PM0Z+rezjIR#oWw|fy+WfjAof&m_NS+~0 zYsp5O!p4svmr{UA_0;R0k8#jWOgeEeZ%tj%EHp1}dPZu&>eWDGYr~}H_}VW&WP+t} z%&LuNiw&q6Ow^K6x&$77kfu=A0ePnxCWE%6ucgfcpt~7NNelTb* zT-L3ZaOX{mOjGI>XLjon8uRE#Xp+PF&xkOzN{?)}>Qb319yvJ_a1v)#mwV8yO6o*0 zb}rI}t9zzUz9+hUW7>qy(9f>keW=qYV-%AV<$Qc@P9|&E`|{g>?8j%SI-)-rG{4iC zvZJ$$^#CYjVd~6WuV0kt#M8V$uNWH{;SDK^BIe^=V}*J>>EE5_eVjM8-1EoHPj72w z4jwC$gUf8K?!EYdqdPZpt;L3MMsL_XTDzf=LdFDh$*-jo2Nz1({~R%3eGV0E-Z>l* zhqp-S_xh-p(@%-8Etj@riVM#QxOCA)?Y2|3xtmp^L*pV$H~)v-b#_oCzy^AI1B@F?y@j_h9ow_q*Qt+eVs%FVMY;DUJ9B3z zztF72pcKTldPv$jHahkB6S+!kslVe6pUDe)TUsZLvlvKSj!liSF*Ljx={nrTH@|$F zbW3-&2t2uB#rnPGJ^(H3P5XgMPf%5z7|)pSGCS#1Ct189_o-v1(_mg)Y(+Oo>!+77 zR;DMRVWC^UZfZpy_RPPfvll!Ougddcgh1Lol)PIVdcKpeXA-byK?bCbZ~5XJv1*pz zCZZSs@2GFN3cb6(EdQVxDaHt&<~3Zi+M;JCRtA{EAF9LlUbh7keR8&4siSKK6?N3< zv>Z*^TtjQ6h(SwhdU!y3d>g&i26;(*XZpL=l`fmp#d0aQna-{x0O>VOOaZOmH33(h z17kf^@Y!hWwM;pS#JiTbCz3OnsqGlPeN$>Yu+970(`05>7J;QVJnee1&DVvVSQ*D& zTm8emsV*nWL|1bM=2qg|rN%r)o%!d1AYgEhpqpXdp|`zdsAsLV5Lb#+%o2xV@rtz- zNt)-+^~PVh>4t_=P?XPMJ@vD9W4)a}W8+ncVuKtuH)@Hsp>$AUXhNxBO377(0!PtF zq-F(dkftIY8-$GS`;5RN_=WMrmQ~NC|PI;jTDh>*1_F9)vji% zDg^~GEl>v%=`1+qIadNdtCdzV4}5R@IxY^;d<@@Qb&}l{80SS^@CETk z7u}_(T5@vsD~XEtYJ+WapOI6~aM)qz9nJ&GV3^<>m&i2jSeB)g<*pjO7WU@RSYf+?zs4Z-p+Mm|l>y1w-n4Y*HW^@{2tFbV_5XS4>y<$U0c&b?s1&t? z)k#&#X;?LnW0^w)nP&@6)}(#;leJk_4D));R$fejt-EdZRiv4ttYIw55q;a}C|aO@ zT_qFv!&vwFdS5%Ae@`)t(x(yScF zuDLY8>*4Kr>4*8+l=Z+-&7`kS-e)wmJut>TO=I;NRa~*+=@)F@GhtS6!Tb7H7nG`M z=&?;zHZ(NQv%i||4fq0h$g1T^61F#4yz8q_0-rylHPty)tu5mLq-=ddje7M1v?;)# z#b(6afaUl(qb0R8v7DOzK#m9GnCz$Rhqol}P#!18NzXMfC?sxqXn)XH2@vr(u_tWP*3 z@EtV4Y3;Oo^IcH)slGj;9xA9?psHZPB_N~p(SuUQMw#R0TeCz%Tn1mg=t+4x@uZ^= zlPYl?T>ZXcV5AY4$;q*o7#%F*>saRrNATn5!10KLSAC&wZbC0^^DccFIYa7?LN4^eOkPU&925CYD#V?${Yq z>0fD?^QLmV18g)Y#^BzxfU_3`g@}t+oES4lw^?Zf4c(RXGjHTy&dlda&)+`>&XXUV zVa;C6R91gG9iM(+*Lm7m>RFT;0HEukY2i?n=?Hl%?>a=ts?PbE@w1-pxQ;j9>?7Ol zylZ02y}-j#9wfo6IYt_IT)hla3&jGGk}7MBmlGn&KmKz-vGAH^xjZIBi`pQOKOURR z_*qfZFMRZge89m7N*6j;u%12u1bSU5a;sYvYY^`Tjb=8n&&Oxt4_U;0TXx>P~GPqvrCC%z1H0FRAl14Z=2UYu-rOd zD#P2(^$-Xt3!M|L&P-{_!N|SfTmp7NvdL^zt+}0}1Q$3fGISFo^ZDLm7>(o@BL>|j95oL6I997`h9@vT#LnUnVvGsM9kpn zrTD(rGRksEx#p@9GSwbh=1UNJSQ2I;pa)eeko( zd;7K#o7&pHwz5E%kt8;MvI_I&j=5ibc!Xv6&~o&mm3eD{wvD3bTI0J7uJ;^!J++b} z3*<|l$Lh5|@~&3!#r#^&+ zs51H2Tq5`<$FMyL65T898YUUW0Lcl+U?`k1Nm zjp4WJCF{qIfn|^rWZbT~6kPL8YlufxyJLrpQJO3;4c*|?dnm@NX@$v=zv4Hp_deAI zLLo<5i?n4B2t7~5Ctv$0-?SItP;t_0JkAQ(x>ig3+Jvc8ZIzWv*f=b&!P>8wF|A&o z8pg48){@bc*wB&2cSd!*D?cHQH1x)pzZ$AX5;b!_8QTE?=@Gy8jZFvF^`69wtN81g z_ujs5|Dbg4@1&Ghrj~zGF)OOfMl-6Quj>EdPUzP?zHxiVA54rI(Rb_7FY3`t7K&__ z662vUk;9p@fQhoT&paY&qNk+G$wPIc;|_gZ?X`zX$C8C??x3kNdL%$O&r>fnIeeM0g`*nWld*4jwavr=SArm!%TCHVwql+;YqypR>U*Cx&RA1VF5^kSg6?Vg$ z*t?F%?hF2Gkx zZf9ZcP)sBKqd-Ym9&|>TB?@vWynDa(7!2&Lna01S_{v)!3u2*il^~?4yfpNX6$iCZ zJq9V9L-;{dyC92zmD+fO|0r-LLY2Qm=kN)DfEAD|4D$YI&?oB$YKd>h#TjCL3so%r zifGyoR}C7QEext}6(iILbfOyFi6ygH{6~T0@z}wCn$ITfl#Q+0nlGxeL@q3eUlK3a zJBB7#UJ&0dKm+-Q^oHy~i57ME=ri8WOZY)r5Zk3qy-WiBqrgT96abMT;&8LI1m2Yt zRQ-W>;_F$rG!si5LX4q&bM;jYO7(^C!bdgb>hOca<|3QJeVvKWJLo>t^q=p~X|w>8 zo&)J5G~;={1D{02<9_!&A0%eTR^jIdL=zQY8L^u1(ZuZzgaTzOK#UB#X6lFW9|e9- zlXLr%9xFr6DU{mYF8vqP4MKBPe1-746U55C0ig0YQg(zNd0~m$?d=4#f@bi8B-?-) z-WlzEmLOgGK}L3o**F3JTZ+MLcgIkbV;}Io3BHth+bdW6{?hZ5!Ot&Uve*fkA7GUI zKO>@D8;Q+OY?TQdPRXrU6|n(^v5xGdxsm2de>x6ZD3nxBz4pEE2ZWb%fl|GF;fTho z3T(t|>Nb#o_HQqRuj3a}#REHg_Tmd7i%sB8&K)EBhAi`|fGotu4w?W(oJKTWN2b#< zGVEsZ^F0=U{`M)SFedRQZ0LQs1&7ESh*d34sH&wh zf_v_Pbn;|lLO^Y}FgNQbh_zjVa$CpLG~iU+JJyu=5MK3qC{=wk{XD;P3CIH{AUOUO z-oNIwZBMLZMsy~q;J{17%qs9Q9D;y*k((rzY>+)};;PZQxsyYauf;IOTrE{7QT>RO zwRO%!S6&yeahvA>*Lmu-uaMXBgRul^?k5n_j;LXy^WJ%4;_VTL#8pW#p)tMESUubt zgw5W(e*Hz0>^f96*pkDOb~5ywgbmmVm>4U+Ty3)jV9o$e zlPm7?<2OkcfxFDG3(8ZErHu#LevF)f9qxcK(qU~MK7)mbPj25UsP^Xb-R=cj_H|53 z87{-POhhSyRq|E?TZVM1?E+*$3}ZYa_8K~=#gE={-)$tJ4Z>IH=18C{!=vtQgb{7dnT7SK={m~jA{6&i(qi0AtzQE`hSUwGL=7NS5$jQuJ>HsR1 zaTqfP%JRAZR5T7}M<>%9K`oH~E^Xfw*k_&VgM&BRfm0$aG4W7$#%(VsO`Asy4plED zfh$~K9~~W_fpy9AMa|?)h;2X_kTyHUNI^Uk1E@6peth@ODMc>1IY=c9-!V)(Xar>~ zESgZv$PVn4@an&X1YkjED`LE-4#bkVudgl*ez?k^nKu;aA&2d3;+<+tQ0#)b4BO)k zDK}elqk*{C0+Z95GrFAk%*dfE5EyAZU;QZESa)9mR-#oC^42(VyUtzTdb-1CV+e0B z@VDI=^^o@<#+l#l2KEwMNJZ}Hr8|~eoA|WfWCHhRL1?2MR~mSfdR((m>v<_rdUdd? zd5Tvs{`i4T4^>>WBP6AP#Hq9v7%f8iZ|D85@8W{O75DtJ31fvR4^In>5FW@uvyQVL{1GBdWjUREgN(JBh5H40NAG%A#TU! zE!+tamL`Hm#a}1(CkIwdOFx4*&EWVr@o7TE2h?icfjR9rf{VZ=cAzclryM3-Ib{#X zO0PBQN+$6#B(|`$vM>^IrD{8m%O~xp?eSY@yUpKXnSkJ&eYHWXY;`qv0+W zaMfD?#Ze%iRQwtQ^|L>HI=fd@5NgRem`it|Zth1^;qv&XvJ#m{b>ZF0&dMZ$vjJ|g zZ`V-AeVtwpRnO7aNqRZ`e(dyLTUz?rIiK5Xb>Js&h163nWG@rbvI6Va%@fTKYot#kPQD#`PM4z9w%@UJ8t%zOI(5zP9mcdD=wkQWv21;# zJuynUW!qN>V*dzW|8GvUb9)M71#tB?&%@>G0;;DsadBMZtLcVC*xHFfTsP z-6h`kaQ4zu7EK2i)PzhWABZIxB+58X_1HiuC$?ExqLv-%wuh&?-S3QQ9~}RD;9ZFr z9X$teEfJvIZGKI25uAbw;Bz<_n7ylB8q27eTFD}6n*+_R^fQb~P%8s!j#v9=EP#sg z3aU3}iM?R&`e7}^YJ@`et_>fLo+z`0D^rIXGq^<3#3@5sJ{EyhAjqdQE{*j7(B{c&uk z(DZ>PAheRQ9>fX+8wgb9efbbTxNr1|V6&a7yCAs|BABe50oUc4KG4-b>lhs5CWfm# z&CAFPwia~e#1|TudA-VFfv$XcKv3`%nxpOF(Bu$vsDO&E8{qb478Z)z}cOv9*^gidEYuwG4-{tC^=Hx1MMgCAD1l+s`vmp7R}(I%UhD`alza0|mWDrdEjj z)y`PkXn%5QbV6vf?ewzA0wmN)zek3HJ8_rf>N(*mAI+SY+hKx@c95;b05U;0|ICAm zF$nm6UK>`Fad=_M|Bv3)liSEXP^ya39jI_64wpsAscF{O$4XrJf2zo7OhO$&ix{XF;z&!)#ULr6hA`} zS9sd*yf#6uJh@=VXTgVKesIPzwLn1SHSijYl|PZP^c)S<*85WOZL6R?L=8qy1_qkV++EX%=qI0h~XaN+Ms)P7jc>+c7JL2?vf=`Jrhzih+*3LL0gkDNYc%Y$@AA(dEUZ>hikP2v4(7MKB=75$0|9hALR;J9z9S%2AT>f@p!GW#k zMF?BCd;e`!DX!@=)I^DOCA-Hp(9yRWhLSZTPmUPduar;NF)t91Nky@zN`zfWqU7{< zN;&mq2MFR$*f6#u=dwug<2WWn zXSr|hJ1%}&p6Behirp$sz%Q8T4zCS`WiC z>}zC~PS}FcJ2|4_?RF|+8L65RLd$a*-X%cBQ5|{1P-f|i3j2j|= z+LukX=Jolr6awSt!PZ{TR2mBL3HM7&)t^cZ1wYh1TgaWPqk2@1%k4puyJQB6ep0E(l!|9ogXj#wlX z)~39@_1xe!P;o)6-%!&^DWETkaqh`m1S;}z;NQGwF=n8Yyx!SfI*8s22^TiYvJMXV10e&Vbmo6!^z35xZ%`2(({JuNp`Qc2(z^^{CesCOozUNg5 zqR+kyMBgOtsVPD9yWmpil#q8lT}O)02Dq0T3LQFm)|<~Khy(e z3BuPn)VqO%w4AB>XNA|GaLi5U72JOM=Te%Tr`4*H(`rGzXB}!7`CIa*Tjo!!X>?V*P$2p zAN$BgA^rAJ=_}!oCwIdkdoevdO*rHsq~Frdu|&k7>xP=6_3dIcUaHc0mDqek)9TYK zOYh<*i=o=9J;koJU>v%jj<7he5N_oG&Za0Wx3j7QQuZ$BNf)Ee+x7a8b_N{l7U7qe zU&yN2Yji8__lwMA7TH(2Hepw^h4vI@q+yzF+dj&R!6cZOgCn~|2~W-%=>INb|9-;N zZIFPgTf+D2zUd;ER#y4*vv8h+%BT8#Ek-gEPvxvuDjEF>wixoHzF)IE{aw&|Nok?0 zcfCytI!5&!HLLZP9*(`+y-VrnPVx3#`}Z2y`?!Pj58tSZ9HH1bVO2ZIvHFc-5+Im# z-n&u{C12w_b{u*Y{y%3dDtEocJ_?Y2aSJHN^4p6CPsV*ax}aCN93ZnAxAIA} zv=3U$Rp)TZuK*ugt0!PjXw9s=`wwt@ajHH~*>VZAnr*o+9bV%Oi-4xIeI{6N7q5_= zg5gQveu#Y=?pEg*{UgVz60RI~5*_P{#vLP|ttyax+2_EzF6MZxE5mCpbrM1nPE&5} zMdgVyHS3VGqBOf|B{B!jJ#$@VSQm6!n3-gox)lJqmbDs-?`_i0lM~OB&O}Y^$Pa(< z!a*;%AE36h&nR@S|D&+2&XL3?1&lzQOCi&QCw2{{I|(go?t$gs{^ z{r@ONhBaOWd|&w?@fn2BDHTzIYGS%sX6Ew*C+sWGJ= z%2cb@HkSvcdts&2!y0F}uNljX2hK8n1>5FM;vXHJ~&}5cMczU3&Ur;uf zFwXO1$kNKvbLXxRAph?rNAOBigXnVZSeidADChuq_-|gx7vM*nmqY zX9HlkEnr*H4@KnldKDF+-m1_CemqR~M!4R0PF2c_58&Z?`FXb>VQeVpPbpFh>?b4a zIUjnscqcoL^QkR^{>qX-4`Mql$|LQ4{qNL z1TSpzvQID@Ub@~rr{dIn!gQ+fk6A1Z{!$~;t~-uE(vHbr^$+U{YCJjwH0y;Ss%Zm~ zV1~pLAdoTO_^Z?4MW58la0}H3CZZ3Ss9SFlus+vC0fFTturkt_X&`#&^&3U_R9gVCIYDZM z?eomjnAPiGGgTHONe&$8TY`|D&9`FOeifN;6#9VpLFF3kD&3PgwaVA8Ur+xCL=R@S zqV%tS8hX}ow|+dHjGFRDS}EIDk;&ek2ZEL=0X$~7CMCg%b?7EU8;M@e?juQ&$FG2^ zf;Yk$DkzlgNqp|W-KQq|AVxe)AV5r^u(!v%#Go5kz_Q4R3+a6J>9{;rx+0mW{zbtHGS8vJT+H}pt?ze+Skc@&=IEWD@On3Oms)5E* zxZlT4c!@}3NaV39cRlgaM?$VA2mUVO$Ckn~VLMY!Mz{fp|0W{olK>{~1WBcQucd0^ z$w>%Uy8uDCztXMY>z&N4D3#ZI(cI>V_-L}dNY2*zj(V@Q^NVfcBj^#zX;a@XF`1vx z{KkfVjVrFB2CN8)JUCPu%2vI>esEP3c+M!*wEa4GV;;J~*>9U?cOUPQp7x=8eo~*3 z;V}5;` z?GrHR6x9=Uhqii@GjWL*j?wcdfu(m;Y?XL1IU%lLKtWWL2j%8~870avE%O z?oBy4vy>#WX0iBFu{LD>Qm|@T_}5*jb3f0dou6Q-{ay;Z;yCClG*Nfz$Bqq#6?&dC zFfDokR^p38f8e$%+m>=Jq5#*zOVPA?-16G?>g}ynsW$|aPPHB9&{NU7k?>f=L2vK! zeHD+Zx88W`%FoexFP>iLqx1d>PyG|>kGA|{aVOt^Gwba?rne%G4I^y~=IC9jbLsxX zV#n&25p^(b8GN+VBB%)}|E<+~C#^IFTa81QCvLCj3Q^wAN;44nNzae~H_jRhi$ z(|S7B)6Pxv$YV{X5nFx{azKF;^P3mIRRgRvVxTmrk-5o29^01X1vN!eVTR7S3rI7_ zcIRR;J>!;l$1I*XHM0zEflG#Kt<}+G`-p>@5FP1JmR}St{YjGtiitLV2p|C{(DmRT zbQS$Qo8|LhC&UaWrxRTLw8OANqu;HpzgL4mKGWmi^_WXFo`_y$w zD!_0Ldn4!K3soBq*_Pw%P?_h%5a^7!iSwY_y_j(_wEy{m1M_{nXFfNt;4ZDW`MZ)c zQ$vzNVq2^eec$FsvB_ii{RzO?p3L9wEW()p|({g@X?(saFbA&s@f5x3^%VrY=F~%#M?eZ&7 zGu4HvC~Pe)c^S&F^Eu&2uTQ%7uJqhiO|w53Ssx=KW00z6Wf4&T#T`+r#Z#VB)LU({ z&8U6iK}|l~_2G|MCQa&V8MAHW&uzE^w}J1$Ys~$l2Y$CYi3@szdD#wWS$o~b(2lku z3ZV6qP+ez_LPogr+)@2|z+(vtkPc1`Z-kdU9=3F%9P{RoY;`@nGusv*WKo-z({y3d zRM0&f1;o3j;B}k6_5igXZ?0s*3Sh$z1fD8qgRSWkZ+p=QeQd_<)5goz=Q5Nz&KW@U z=7(#J-c?7E@}bd1FSzAbO}bx@dbMil6rds}l)SfSn*)MY%eoHv*kkasay9%+Q?g{8aF&UcSVVYlu^|kGY5QAn7 zk>pnV3tR_&F5TWjYY%;#HS(qBIo78s9GPuT>g)fsD%e%~Df*<9HczIN7`=Y<^>YSsCW0^Jys;Y=R$n1K0)O9`fAPnx4Pte zAZ6iY@Du2Esr*Lv7Ju7n3&h!zHNy2y%U2S@W>5WiBwRiq^CHwiz1bABlc9j&uWfi6 zGZ`L1%-W;-qy%l{^8idZ%^e}%^v-_+iEx0M;=V|FR%VOdan#i1=B^AO`hDCWA~%~3 z3*cuW3!E65E*TfU?8M7R>C+olf*d8w6#JJ>He__J^#1&T9i%PlhQ^cn+ zIRtK9;wh>funge(gP7s_bDRVxrU%6gZ5p56Buv*GR+y7Fehv(s21D2l9KXH;VK??s zfB}9}^)M}Qk9lDZ>PM8xP)tLW3YLB0*e*)KGV@>#?)tOWS@B!fU<9Hy!7rgsOG@eV z41~a?It*I<{{EkzAi?0q8gd``Sjg7*xqx>0!P9qW{y;HSSg`96Il<~5=7%}-?2%$3 zIM6xIU}^5vfg$kB;SPb1h|4-;Mi}NMWCGYJOuzqmUDRO?q2CW4L-YTC++I#0a(o=V z_P{Z4SqBe20Gd3)Vt|LLHI3)+E<)Z2M&he zJKZ60h$1N&>e5O_-}>rehY5b2T~Yl(N3`}Tgm@9#f9 zD?58#Gv~~laU93#lmPZmoJ?H9e=|pngh()|nFA_43|!n(jGvO#)e~6?G}=_2l;UMP zC+g=ecJJI{?`2*FJxpq!|F(A@+6P6(A0rJLlwF#Zg(~Lp$p6|SSl2ob|JLnj`q zV#L#$y`Ox42ZvEBc4+o#hU;%;#o6esIcC-bSH@TFmQCpGescp1uj2 zG2X;k=s`(_40${!*?yr=j~lo?Itm?0x8?i+{+}aG>*-%R02><%96V$zE&m>iU`v?Y zzuO-=p4g}PfF?r2X;?f(%%U)52K5l=n)`c#Gdz;19+bDF&KB#1_JXW}(EF%%zFjrhH6=T3%qg2fnK$Lz9z}dntHl#4GD~{@6eo)Gz;f9nBlSh9pg; z(EfWn{ddST-{t_>Gk5>gKyGoY50###)k4B5(HF;aqzJnlp$9MrdJUKb-kYI~Rc|c> z`AK=pB*XuPyn`$O;se1a5YNBy!X4jIg2*#Qd(9aGiS4zM;T7Y|sS>Yb`v*|Z_x`+_ zuev~iEoS4DEwh=%Lt@bW>{Z2s7kmG?^&mX^ps4eI!m|&}uIN(CqkljadVp45YwLnn zi?{fWW#|RX3JJ-o4!kWC`BI40aXvNPl&pXqx9@kG?@XctSpNd;7 z<2}x&6_d#su=L#5^u{eYjA|qOJFlQluQ-RO)Bl}S&W|6^KTn4&@;@meYUlP{G*qX=*CEae0H@{iCTAu1Q$H#Pqz_=VGY!6Mab*R5dxJuph~ z*?fnp=){=tOw>$2(zr-}7_XA){1rwe3m2GvwnB1mwT;(op?Y#Z=X?I-5Zm)MWQ&ES z-EE)|Q3DL@Vxn?)M$o!9=ox!?xpg5rh24LK%m$k9j2nKCefamuE8>K;(Z=>`?r#_z z(I`tdZ?#I!hF05GEgN7Rq}p!DT0o$&?|Y+t26nc7aetK~byJDQ&o=g4gj5h6D4Zsl zB*z^?XwJUKCz++tm@(<|L*CH1{=KN~AaNlIwCKz&f8EF-StR@6!EmpA4F(>#$=Xt> zvb2-MHkQS5eqkw3Jzu;+)K@6$47p_Lp~J(ZMo&(AW&4+5Dqo1F+xUq3!XO_jk+gyHleMZstsp6DyzH=g>q6Nb*Qdsc z^sEr_DqA<@`97Pc=gdzeQ?!G>XUU&tV2pZCAN296C2VJJr~O?0v!37G$oh`j?lEQH zf2Z;Ra1;ha(Ruz(-d+Zy@s-0v^{z&AE_{2dg(~OvxJTdh0;W%U zhaho}I%x)i;vh2usjB=ZhZ#@MI* zoC6EXj(Xo3!^kIzeMpg?dm?$^S88;kV#$=Eo7V|X@A11#(z?Lw>+P!QK@ZjG^=E( z3tR@|pQ*fc>E)C)K`#u^Y8QdurOT);*yoe6Q3Uo>URic zqk7K9rK5O8L|i>-S2003>yzu8-Q~^CQ(gwt?(=itmXpQQeKUd=>eyYOVW&S+hyRg<*&Lyu*f z)uGIz@vnqTm@}FJs*l@YFo78xT&enr(XAPpQ)!pEG|}g`29pMa_VSY6;Aj5hxd-4& zJrSdyno66&Vf&fllX_)?w}(1SGnqu=eo zz?kBNW@or{5`Ab@&3-XMFVuGZPc^N=zSGu|(_%-RkZb$dtIUXw*`9BO<=?ydhgcPS z^fvdlQ_pJcv!l-vUs2@hmU}!;ZLOI(v!50G=av{^MSzPKZ5x7`hySxvA8_@ zgW{VBpV-}YeXwk^++~0IPZCbJ9I=Y^o>GV_$2SPfmhgqk{@rZ+Z79@iDai91c+B8y z3XVY-`CfH|Bsnp?8AwE5C3(L+|TL$7}CoX|)Jv$S=Wk z%c+y3WWBG^U(-+E>NaxEp=IWW#ezNyID=bS)`)|8*ED4+p_WB6F^Sp%Ko`J zj}a$%1PdkN&NoB$J^S4cF-e>JOl2WU6@_BF-=ZCrL{o2X)Kg?UGkt8;Wjvy9&|M6J zx0X+RMXzV@UNV02IPL2|P15(wxe>gkwacTDby_~Mp>cQ(reSQE3OXt5Vry>aKfdA{Z*%Ju_9yukY;(Q-ld$&Igev zK`c#o)Mq%W8%a*;<(i9o3sTFU&b*vS6uiA9yl8-L(DjCX5CqT}XlI`u&zz#ZG?~m1 zi;fpT9zh}7_YFppb4)VPdq#=vQn0NJsXqzPK3$WX8uflGwsBTzuvBjQ=TAhvS#*&k zL{NG?dZ#mGx3PVFK|y>s%YJXz8MjHbhP(4sDsQ3`#pbnaS$PE*@ z(K74FA1pljU87GsnJ+y|cV}ydH5P=*tbT|4T*Yq`7+suPd(UY|f#dOPd40LdIXhx6 zgkb1LCUj1#Lo{b1u!y++d&MA^GK-$br)D-JX_#eHN~^Sm%;@ZckhsHW%HNZ_C zG@SbH(q37aRQTA2rGB#z`K)J1CjLzvd<6oio~7wE+kQlSc4m6?hjjYgq9f-t_3h22 zR9d{#=!4(#4*QJ`^pQ`n{32|%{=aF zZN4&CJUb0-(&Mz66-~21QtAK3ZD0U8^QzE)`3J5-LGc@N^1AW2lg97wUW*6d2l4=X zMdy(;XdQ$6M2;e&WWo=ty`z-ouH0s5y`{*swCE}NN^JxHQ8${IM|>08aVJ*^t%A3; zmK$buEwCmE&5G*dBTL}r;ys{Kde?r7uI0U}H+FJ5^UP-SBi%z(r(*IUz{ng`m;c~eyXh>pQE`E=42s9}kpJR}50J*ixIN=0h zGmSk@d#qvRkcaDaQu@EHGBw7yJ-P;)b{}cAio(r1cUgA}IYoY*er-l%&$Ts7_n?>r z;kT{;#itKqmU=ZsqTSzr^gO#G58e$AJU`lf{@e2ki(tI$;In?Lc%Y*f4cm@)aL5>b zQec1LgqO*;iaAFwBbijnNO9qJEhST9SZC#E4*$f`c~pv7p9>ooKBZB1iBd(Bgm-?BO%D3&71O`#RkqmGlg%#s>=%g z7x&1=lD7^%n>SM92QP3rt;yec=WL_Q>nF?#7WTC4*H2F5I%)l1I|{3sq}0geHnEV} zyx8Uv;uL_zdCTkJo?sj+9UAY!f`=K4+bvG9O^J8!QX9?wA5g6!oqnq`*|27L-gv43 zVout@p#PxkE5IPWf=JGmA+eytX>$oGM6;gF9Hla<$7&G$A(4_xKyyi;7}hTT+c6_# z#3^cuj}u(wzZatyiKG2#P8PZ48C46>J9isIb;MDTkbjO2uMk+0Ap1))Q6JLKA`*^ zFsGpLIi4~aEDdhA@M8H#*JkmB5|1GO=3JSUiV>9{-l;6(e1+r5QG znXK|LUf5k&nB!$-QCgL`1i6x|R-C4H&B92x0gmq_a+#k+>~;4bUfe&k{t`)JnZs}E z=U2=ZqG^aTF-cb>6XXksUQ^@~&pP2FrW^NNt~U%X@SpKsA!!(L0#FShmXc`NA{dWESC|Hj*GWS7>RV8S1z^$xe#pK> zz%$y_42mL0JNhh4_9anXG2%H|8DGo?^pUpM)4hA{pqrYgujUNE-|;z4;k0u=pMsLB z0%A#u2YP z2}uW>zkb-ozM|n=uT}7TaF79=NS{!Z==GG!l-Jr7wW*mDyL->q0^JCEkA-emkh%ro zNPs1lpvWgRlyDw>9daa`29%!ilHL+#0vjX}p~x>MIPLi@DDSsigda&%m?dBoj4|3N z_N`(9%56WrLo09U#jCViGZ{(<9$7*ESAj)Gx1hSl;Ql3Y!CIs&yGO+zm!x zJVmk0=O2Xw(TtoOqROPZ=z}m^bSNk&H6a8sy%?kKzR<~UYo}_AIRpyRVGyB>NeZ8y z6;{vlaUxZE)R9+q{F@s{6sEAm>T)earxklb0WuG0kyLT1r@5p(^8f}Z08N*m*nY8)CyC@Ab zO+fPZf%w+hd4XYj5i?)%Cyd?Ej1_;+9yrud} zDM4gS$utam%>;=w8=c|zR4t7g1zrRJUT{7Ri+y93RfGMw1TJgu+NMxsEi}#u;fZDl zOXcjSxlzeKo_rmeQKcH%xU&+yHd+Oulm_pOd@7u_kDP~}&63ggfCJxZn?nCvIja~G zS$Q$_Km{AE>^6_j^`+E91&8gy_z5?Dd50w9!1%9-!^!M{ZyAj7)YY*kG6rD`8+-_( zieo$fBrFIFcYE=54(^7aZ`r2}W;?Vo4JY2XxDTwnmJ22f@DsPTypfbF(efl`wkG~h$a(qVt=Roz8UBXoZ`mcJD9@<>rfIx`IL1zKeFm)g#uGvMyDz%=QLJe z5O1W6sviOcc}=0WMkh>;xNJ(tesUlKK8KXFZ;#Ctb^H@Ba zOgxc0Qjx!Ml>1qmr7%-LD`2Hs(I2Hc-Cq>!3GU_+*_aJ6DowYq*CCtt;+#f((Z>YNZ3*{uv1ACo;?*mD?U*OKx2GEg zB6}q72a*&NIAARVV(jkurV`}5`<>(;`O2O!WjE#U{zF(2@an2*CmWv?a&GfVhfcgK zpY$G=IE`$bRD~c&IkreDl`|ECSB>G_2-twH5;*H3L~xpLH(0@-*Rg^x1BDsy5@2ns zkF-%4?^)8+Rab6zRap~$yBgUSHxFv%J292$wN80g0VSM+pp6RQolxHKr9ncQ_}~8d zTM7I>)!5f~B`pO4NzLkvX;=!MM~Uq;oqC#tDi$75PO~DdN>6SBh|=TW;p~#Cj)A{c&(@CL!>1k3RgPekdVhQFW8dy6WnY;)9bLKs% zF;RM_m?WN^uI-*zfw+iEqDB*AOfT-y0nYpVA@9Ih~c$UxF*#p z;v$FwlTNRMC}9vu0C$6kazHd~6CwL%bUoaiz<9`h7Rda-T;GQ52kj|3Y_nJVfOPuYgtGS;r(Z`JW4600Ecv#EX6f zZg}wggaybLzCQ_aEPf}%5};(78+Drg2`QTXlSl*$AvG6qO0Ig5E8-LssI&5GB>b6< z-xC|Ks?zv5c?-ff`cYX@w-EnC#7w+2e)XDShz?u@){HD87`~KX%8ls6moaUtpeqmR zx6tFy@~q?9&2IR5n8dm^gM;i}$lLQclWPev zC?GO!862~Kn(Cb2P9)0hflhP}Y@Qe3Dn)aO1h$BZ`IXr@PWaKi4qj4%1>EdqBMFGA z+fOs@#8A>eRF*T-$$%(}cZDZj9;9m4p`$tFXTjfl53)R7b%uuoe7e0Obm%A+U*TvL z7Cld#1a0OME-IgK&faE@D8Ww=5@BhN z0+5akXa7LX@~FdWrt&xDFh<8tNTB*a;66zlul2cT%yTu}Glb)s=3Z}=Qpl(N5zsXe zrdcOLVR`wQe`wBu-V~SUD}b!zAn)V&CZZd0?TLR=HNB%$gjQ-*-B`jH=aLMwyfl@I zROxFZg3UxgR4@&nif^bB_3R0*P97}8G~g(3-KH)%p~6${+RiU<`jYe-GZ7_S08GtHqfYm+;_e1jS zL5m^XI7esuQj{x_1)ZS#qowgIY#LfodSLa5D5br}b)sdY$gRUNIcnqsB5Mdh-7;yi zIuYtRwh6e1DldX@my?R@yyPbgN8H*?SU+C{qM6yf2PD4Oo-gRmpPjPy#8Z2)N~i}P z?Wmtus)WjmD5ActpML~>Arnk;8y(`tWBD@sQZa~s9{=~q1aJ)v*oTWh%vZznWwfif zIf(Nt#R;7R={}kf*NR3kr58Frn&ofe8yDS4DF^ukh)A^II+h;y_Yxo&tT(N&VCtUUtU0OE!gI>R=XA~Q&Ad>UjEpOI5N<7h^l81BNW*moD zknr=zQE)gw$(VS}@9;Jl7_tN51(M%Zz+)ECmS!6k?Ra|u@1AT?m9IDVh^l_hBa7Fser9Nh z-t{W{wtT+9=PLlaU!GU*{U{OwUVR(6Y|H)l1`m92AAAr492K2#l12(PcHe@486;DQ zFJFkPlLDPJvFN@Xlu(2X8J>s}Q7f93uP`I*M+lc_!e$6{t4&21VWiIqy7Sh-SlS1m zZkYilk}Qn8ky0^OST)1?F1s)Xeunj7NqE->UQ&AY%lPVh0UWLO0r)4i88iWy1G+0a zWB?sr4Ui83wiV0(OSgKj-IN*)%aT7{W$yDkDm=vGzQ*f*5=f~WClXALcJvXfKWlJ; zEO%IQNd!9O8@0FZG*U7$iQ+et^88J?|7rn<8Q18L4QP+Hy<7qi{djr(mXG&$f%u&B z;aY!^%bQ}XnNJ<&^rdF^eGOo!){;!YU&IjQNhz90l`d*!da5>jBwRFbI5`pv>mwnT znc%*7mR}@X#tScPp3PSB^dT}mBQy;L?2+bLpj#L~bqsLyiJwI|^3*iVX{*ncLg-Vw z%1A*lhdUkN5<{7bxY8m4(VGuMF(AnR+JlU@q1lrmJtA-gqUgyc%GtywpR>L&*g(0? zRLrnGgm8qxiChHRc;!QH-6dZH$Pxir5KS3K7344I4diQoWr%KsPFz75#=?z{01oIs z^DVeYAs!dqU%Z3#gBDy8GvDZ!qp2dt^_k=fy42-wb1w?uNEc31KnBF} zB0Fy9Q~{2me$m`1o3d%kp6V%M$t}duVT70FJ@~EdC2&vvt!~YgOyyAgn=HWneOH+K z{dBRFP%;8T&BlOKq1Vwpn)+zjA2>93kd~>0Jyglf{kATCRS6k6hZdzC@9NVYQZT)3 zVKsp7Qy&^1rk&EjBLb6n1UlS@JU;m_Y#zWM-Z9!ZJkb~m+J|fP7UAF@tD{E_Ka+%X zG@_SybGPaGI8OB}uHL?{O>XENZg6{%`8xXw;|^^_gxhvy1B?=;cH$o&XE?qZ#nVvo zytu|^;^>5}j{~Om?usoOo^ZlcKOn^?<3XsHE{NH+sB+3m96>v0j|npyL?T4LP#~S` z=beW7)AHA-RVAAbqP$e~@>A5|8h+rP9X_mrpLb(fkXp8U&k>^;I!5t~&HiS8xnYrk z*`N3n>XiJxggbX{>`zaUt;#nCTRMg&Ycz$;!6egqmlb-Jrf4h$d#xwsBWiSEa@d0h z0?b}~gIDyLKTTrb3E@iYZx>=uEZsenj<0?NoY2{7MW1?)amDh;T9T~L=Qk!RyS|Rg zdcox7pvL*%*Y%FC-Z;Bj|F|)0a(YC_QDItP>OA|W4kKg&b`Y)NBhM(16>=i`Fse6u z>m++?Z>y&9P&dR4?jyZ8oGR$jKYmP}UiZ9=6z%;5!DX6xB1T@6eg=IV$H%7J--AyI zGM5z^*I%5bxAGaL-*oLU3k;B?E@7HJ^S{e{1bukzEBQb{?_sJ%cjytr<-G_-T;1@a zpx<2)eQTzkZJqC>E@G}{7tOi9yxwhEO=vF;KVlG4IilR9oE&`hhcvZuWYIWAVg6{- zz4+oHytP*f8qD(N3g4e+- z4+-E17SvtzUnTW8Hl085Jz+{9hpD;T(A(ZrgU#*2rQYq}Ie4Gk6v804HoVw2Q$x*72nvAw> zze)WX*I+z{Fn913#2>=qUYEqD3u<*U{Sky#hrjJs37k6GtBWyRrM1=I`JsLAy;#Q0 z23{%U@g2Wl5Tty%s*Wjf1X^&Q>Vn+SqwFR)WjntXwzz`*73w}--CRW29@DV*x3{s@ zO&HGl2&8^n?6~S`1HkB3V(|!rpg_wm42YE{%ePQ#d`U-}MRTo8?Wjw*!KUZ@QSR(= zF?yCiG%n+bla8u+XX_w|?)(u$Bh}NutYRzI*rv?Ae0jb2&X1PvCRxR@uKS*yZG&0^ zIyE;NL>C#8A#n#J<>(=B2 zFz45}OudjHWglyq&kI&?sHY8rYBBas^UaU!;_>ZsjF`47fz{g zg^DP-ucsWO8AhFrEHJg>Q$v#Jqy}rbdAQP~@;qCibtJUl!U33#PHbnv(NCRWPHrxN z-ouikO#xV(8EsO-sZ8p205Em?SQePZ@hLq_W8zVDNnvDu!3D>lU@=<(ESFIL()a&< zH^GdSWV+nwxm<-fA|TtRmk?F9s?+Fuaa##hDZ)i4yWBFAC$L@xp-kxT@-79>&E8aUDk#+8-@cvU42V z{5;LWG+xXK)O+4QF&lFkhsi4F6;bHREzlHESZRDNr=VB6mnXdtn%0ol*hr4a%I{T& zRde&voqpB(rxwI%^BO+zT(b71T#}UVYghjOpsY27N7ZEN$)IB8N#;4>n%t)9s<(JFZU|&Z+g{+cC7=!L9XBFBW zU&y>`_X@^7*|qH-wC?r#R}bA7E@B7hH3*U%{W)ZsOjU_)^j|LBIg)d@Y`>xxeD=c= zJY}yxkNX8t1aX^t725c*9ZW1|i-D#lPphh-C3E<9cm;@4S#ES?TU~>(CRX?VG zP`=c+hY$=&vjP8qj4zQ>6la?+y1r)Vy#K`&e4GIQ)q#(hL;rDw?)cgR_UUu^rzU^P zptr11+E7Iu)&j+td&9_I^e1nUZY)k`K#UQAfkgLHzm;KfJu{)tRlK?Yi^ljk{5wdpjm|mr~2u zP!{(cCtR&aQj9>s2YmfIUQWJEDxp#c%ho$F`1n%D$4u}>1oVn(a{~LJn!uTE>j0S zOB4SQck2h)M(3>y|3in|8$P4D=3Yp zdjM-#ogpRoOsBHc4t)IzU*WILjwA>CyPQgVboT7y^^g^l7H;+m=I<@pOoJG`4BUGG9pO~q!uzJ`n~ROGlJ2Dv}srH zZbw(z2BTM_?}zmigi68IlYWW{VT1J8T{Hm+{joaY@F@_4l%F46>^$RvqMRoN5%Qco zNzi48mUdW%r#r6v%S`9UJJh~cq38B&5fPGu^l|7Lo(P_-{vhjzCg!&H%{^%|DswC4 zk4vT|5_ydbYt|wuX%Q<&C#fG(=jM=WqtJ&o4r9nK&31}OV$C&-q2aT2`5l{k{@MY$ z6;FWpQ+E;Yr?vE-B{a$WPx4vS(9eO34UGFFpLf5k@1V-K%|x5}&12SE1Dc~(tg_h^ zO0aPFtf+(CB_HtSWG&>qYuN;^3_%`VF29@f7Y491k&ukH*D$`>$l)3j4|`XB|1A9s zOq2-XQ$mxEKk*oF#@c8BJ(Ocy}e=Ye!9=b>v^!z>pR3Vn-d3SMr*_|B&%b-k5#ze#3t0Ey8gt%s} zlk7lMsveL?T3E`usdhoXEzjo;0DwL?YYS9PC)}wB2QORe zoKV2qBAzf+^D`=4S_nU7RJH*YUC;SN(^7+jOoloLm-b2Us_xZKj_%yx607u~qXxr81nQ!I;iuSueUm$Nr{}VPs%Y|^N zJX($-#C&!g$U&^@p4Lrv6lys>9o%Np-N!h+xn8*GmAwH>z3us%>r+V^KvgBJ2EDc< zyPS6sdJ#5RUsW+OTw#`a+ujG!&@ucupmet}e_ZsQHsA+PoT%rTP;W!X?7ScJm_Njy z)7QX8Ocd9PYWf)V(9094mnq6{F8QAQx`BwKs*&t3K9CS|ftb4K9{Wo4{mb@CtbXf2<6@x}p(Hd5S@a{AMHSk+dwP@&nM(mYF&oek zFzC9}iW~8J<&MfeTfyuh#d0Y$eDK_G%yH>mFCOV-E=WVsj?ba{6j)A2~{soga~gUm4eWv7l^%l%zIvXE3~&BG~L$k)5F zs0Mx9aB5C*$h?k}PTR<@hf>_Pa(4lhFxvy@MYMWf=Ib6ao+2cJ5VXz1lt8#Qyb6df ziANiKe~HD+X)a;U3|J2L3=eF<;~4WD#2KjpGE_TF(zn3z%x&zB>Tpv9GG<)#LymzQA! z2)=aqbb@eB_>SK7L7e0Xq9qEohs(5hn%UfJ(d>Bi^tZlC`%!oVym?6M+_131KDKqg zjr55b>eB{5m<-!WuBdupL*DT6jrl+|(EEFa?HezEdxB77$<>dNauQ071F-9>%Ha|!KZLQ|7fnQ z_@wWku|l7!aO0Y6SX@ytVO_lcH7pFskL1;5P+wvpnV+f!JeGe%(Hy#!=F&z(@a!U{ z8+uiMdnjN(iB-k?exEHm^?*K0m*e%V#7Z`2uOs02MOR)&&gpnZ*hirrw#zW2Y@DkN$}p(6fq!bh4JC zXv#5>t4Jf3Ju#~k&}swVuacHIdzSz?4kO~2b%UZFamblUM*6Ool@qz_h4(=9%1R4Ex;)jIArm2M@MA-q)d3+G zMHD#yaAF3E0|*Xqz~0J0Xw0-Nr;Alg|3 z4f8Qtdb(DBju2=GH7h}4{qX#FhDT!nArSqRwRR=y3&{)uhkHgRN1+@jBb;!B3OQ>r z3vi}Tu7_1G%*$ulKo67GYXD=HMxfY}u~3>0cOk_j{r%Jij7rCpJyEU+@L>Jnmn5K~ z?RjM(>^LeM4sftx8LyFZhAY+u91T;ZbshxHYGW6P636H@DYO4f|7 zTuknKBv^zQjyp62M4g;dYW&t$4JhNDt|`dh%q>h1+LDr!QMKgjZ?Ek8Qx8p*QQx)P z!RhCn+^J6Ecv^b4aC5bAmq#4uW4QW5RSM02S&na!osA-7dwA+ z@l1?6Yvx==sjW9Q}Q4Ll2(5}j!TK}<0=Qr2p)6w3+JI%sN(DQ zTD*RHQea80R{bONjfRT@z=wJ7q|1@;$vElTp*P%QIpxR6G!x&SKVw`Oe6G0L zlB4xk`bY}OLcWS~Ntbcf(LEQv_o}4klIaCzvGDaJoKeZR$G-Z5DP2<7&v%_B!{HcY z=Gotao^2Tl+)8X2V2JT#PmQ@>H1sg$^9mZozJyhbCig!b=QC_N>tJoo9@hNBbRDDZ zU7@-Jbf^i7Jg#Qfg?Kz)3$m~eij>Q;HlDqE543#hff%5mc9KlF{b*mx`}s{E2SM(q zP*cgBU=oN*8aH-KeB6E&$}xa8$v9e|CTKia*CDM?|7ac5DAzxROU9EGDtzba(gmtt ze1dmR<;cD_dcY8r+~IFcafY2dS0@0y+tU!5hQ8u80 zc)X{6VrPsi4M#TDlM}ADn1rso?g*Mcs=4M^+SRVcmcLz8oy;u4B>oBmDdMO7TSKK2 zN&G<6w@qUp5ii|Rt(^U#UQ`0a$J-TqgD;ju#}I6*^NMP$okm5X#PJgl^EqqUUFgU5 zs7hGQrhuNhqqE2<0d^Ast#5hjuCJy>5?j(QB+J(-STpRLn}YLA>^3yfblorqf9!|Z zVwpO94gsG`D?Mee>xy~2V76ABGRZsw_)B9DQrN9d67+I~zHOQ1qsPFd*aLQ#OAWRI z`Wux5aHgp2S1}}i3th7ArDy{dZS95~I@PcebTUWuLf zG8?*OT6mtTVXGOf4Er!)HK94_O+pS;(y5SByim4fNUd<69{eQakfY)AI(nM=*uQGt zecP_nVB*QR#kNXUewa$qb>d`0rxy^Jdaa$IS&sDSl>t%V#k*L&V?EA%#RYS;SXBF# zPs=3>=|Zt44v>1aje5wj$q(#Bs$u%EkO?tAFJ#EHB`r?>DpFEFTo&f2x6y7g-H&BU0*qF_ro4YUraa@aZx*&r?2_xJ?E;KKygAq%IEv0}d?rQ2%?yrSlJ z25#J$e4AGx%uJ!=YOgXtgH17I?V2CRWsCte7C-Yv)M911`{YGLcF?%oKhm4kPRQg^ zi5Pr@!=)ya@bboXIwD+&^~$yK1fXqT!-zYV;YesQ^fPhfiM0v|q&Sg5e}9v~W!7x% z;|L{Biv#`JSL;->5ra7s%>Md!gdRDCe%oW7)&C&)NPr^f;{)B0klA)vgDdxXZjXumgP{<@rHM+i;V zJh`gTb;v&G5(THUbua4reoA_G$`X!PD2?gp&dR_@eqn3w2you=L+e=#9vP2io>(S_ zUe$$HjrebS(N%sGpPp!ty>d92>G}O?p*e{`H7>_%_pTVc&47P0d{+0cR!SFE+7b23NOBS*t~ZK7)H{U!;j%TBS<)kJk8R~Zh< z0pXQ0g+$;<%l775O>*w}P86JoY@}g`gDpQ$Tboec`%?T&wu7MwA&8lAvNHF_yZR+g zwfKeXJu-SbVK~>_4KDG2Hq{eJ6@w*9Wt=2m9r0v2Eb76~Eq$uj@0$3|?(B>`2Dd>@ zaT2|cw+}mR9yMzVa$L?X269dJ{KD`|o95txo+aOzu+a*uI#t_yp^{>kp)G^{dG6(E zTVJqxq$@9Ff@pfjD&O9!gt_+)CaKv|LV=u=HQnovfn{%70(V3q2&Iy5B(8}6+OI#Qf5usEi>M)~z14k^6%u%tcm$9Y%b34V(R;jP!C{Jq6BsO40x z6{s9TR319?jbGBNy?30OXu|7IoJfNcYH>{F&R@yh=V;G()i}FRcf>EV^#O$=jvC_T zXW^yZv2;kjE2W6jC`^`uIZ^lE4>dvj>N>(-CW@=7%a%QweCgFE=R&7_CQ{jvUAQYs zjDBcB-Y1L(+kqlur;Iw&qL@ph_+YkGNl$(_kAu6KzP>(i%3%%_m;qN%DqH>9_h%V& zm3qs}m`z2!d6V#2RYGg6L-ym|m-DK5wrQy0?t3QA5mhb)jUT3Q2+*D6aT@w#tO3(QXDNaP^GgcO^HF(wk;RkU+RH2YG1OyR{ke?T4fn4ZVATd-(e}fI8qi3->XJs z8%VjErs=?~;EirlC21(;XuHa4{mS|aBzE#_P|rd<**f_kX+}qm$NQ`)b<{XW?5@H5hfLsswkZrYdk7U>g^m=^8qU5Eay+ppF19& zlO{0oNqU>gz}@5;D8t~RAH)?Z3-KcN%sU_Fjo-Nhn&KEuiBFji@%V=I=P%ERor%G1B}r1Q z<1AJ$hET~*W=$D6d$gUw#bPRgktI_DKqIW-b^KR;=Qkf$O}=HGE6j0?wuJVJy+xJe znCLr85h1YlQW7yAD`4e#XKt{my+^6U&y~eFUAYeqL!P?lpVVroTZBQ9+z5Sqi^B3p z<8PJCXRHEZSwmB5*-y&g7;*Kn=@hrR7On)Jjf z2gWCUYt~v`-;JkVk59HlJzO+SZajkwDH1rY zAoJGp%C3W`w^E1d-Hf4@If9q3(!`l|mWO3clW<@RK6WZl$UFpl`p4ZY4JUSY89A?e zGM_HEO$z2FY7r_(CZtvU?#;H|p6PXFPzDUAkNg#S1b%u3W8yO$O)O08F0W-ANO2rS zmn`b8z?SNKmlsFNoQNKa_W7bKFaPSIdzQu;EL`PX_fL~sjMn3Uswt0{jbk1vo6#?h zU7i`{)u2|=a(@w9R^;%8f9F$LmdhVhd2iWg@cntZ=5m=WkfR^9#f<|`*MT1#c98V0e(qHAi2N>Yl)(26#;($0I4EEhTaQcjFx8G?4Yus23h zf?v0)D^gW?elR)7l@~o-p7}ym_q{4ghE>Q!e@6SbWD`QJ|2WwVY=jPe$&3aELa3Y$ z4RaIaxGbrP<_YFRf0&p)*PSQA_qXlDm1T2cg`X)g*;xI~AkOE0#>s|AwGmv3mY>#dYB4YK5Q&Xv(Wv_!V?{oMP+JyE41twVR#2G%R!|x_8eUVyL<@|fcWpO{4FEzn+Cyi{j|^=&46sRFO1LwrwXuViMuqipf~HPng`6eR zGI{zjdC{h?b;Cls`>)n&)Li3fWS#fo+vYwAjDk-?NiYY&^>rOaHro|OZi1JtNflNZ zMEL!NF1klt0w-AR!|=T|UIjTUw$w&Nor*yt^Y|lxMi`-GbH5ayDOc)Lkj@RO4)fAE z5w!v=ji3tE_d6A%@}r6tOVKH(^op{M8ycDD;|||0qEM=(8lMOp<7j_m;fLtQ}YOfFfuTD9Y!JL!B%|GyE8~CrxEq5Oc=H5UPk>28I zI)a#xPz|n-mo^`5S;U_%*;`06$mBy($J>xd$SvMP+a3`XoTE=|jc)$_keM`FYh(y#%V>0mUnVG5}#6dKK& ze(mwL%ZJx`GU$c{J3qg-0^NAaJP6z>A4U^1_+qs+L?-8?OJF*J%4S zRb-g=g!&NAHG2#w3^ZN69X%)vG<-qS#4d`s3Hi6N>UIc zMi)TP ztNQ#nRYwmz&t7}2x#k>m%rRU?p23L8kG7UOS9 z&c@Mcrh~MqT1!rDmRy@xwq|8A-CJGLJCQ8m{f;;bmZQU@62gJIQYzcIjAKb!W~Bs2 zo3^1`uk1#DtfE+Z*LFuL<0#Yfr}#Hpf`5mPFHKPCR1PVT?altdOp&W3cK;x?cTzuY z+k8revKSawbS=KikcI1htG!mWSSK@Dq~3U=de_g!&oWiHS6dr%B;9;H{u_syI5|~t zd)(Hauf~G$HPyPv)gM-fPH;39Vk7)6uqlwLoSJp5wn4zsLi%RG$!>*nlkKo?LukZb z!>TH)!T%&dukA2U=AME;kI*D&{{I%lL>emw`p{G)VfP9JPd8tQ^R&^}HElqk& z#%g2}CEqLN73h3;djkh`8e&a|<=*6sgK|6ST3N3R$a)TeMnLENH@d${oK;3jOra*$ zcL&qxv3sQ2OyrH#Z%MVAesvsoAJG-BHhN*LbVO&mO1xU!A;b`_y6&GP$5vpvzHW$e zKEz5VIXhxK<+EC=w>v2>Z2W4*Q0y_-BC%?;+N%<(v`SFm@1dwh$yLQCz1yZ1RURlk zW6G5xLtYtov!FVr6I>r|R4WXN9O(P5G^_UvrgbV!BD?iP(#IPjpkXCnIa43vxcwFL z$PFz$*C~4FuKGhMB|V8G#uSzeJq62$EyfCWT3f1c+m@YadQ}IgbdI*-hab1KeAh4z zma(;!(^NQ0RJAjf{fr2OsVW! z4@pl6SjopXiNt3=DperKtuO6QoGL0#*I{L+$ugHK@;QLy$wQtii`AZ0jb^rK5AyS* z>=In5#8<*ts*G}!2ix(Ufw1J5x!jk=eQk>|R+5dys||lw);fE1S>@D@$tlWyYBX~o zw}*^u)u(ZDL^=GM`rv>7QM^Ty`B!@C&pmnK)vX?Z4ng{>-;cgH57@2{ISljKCE|7) zP0~~k@XAl7#C~8)z3tck9T`wrfr$z^R{{9!9p?!Xe5&mvD46GUNWV*APZ(z1l%#s) z>!50lu)=MPdHkZmV>_05Gs;tc<_*8Uzsm>0#(a{Ccm=cc3Bs;cWHi<4 znl<)mX}6#J$M^&r9Y{d;1RC| z-Cx;>u&r&yCor{k9!rZil6a5}%?zBxj)X92GE>YeJqylaoyyduly}gbjnAD|&>stL zK{x@l9!deGH>9KOo7y_Rm97@tDERbK+oJlP%}_`L$)+zSq~YUwppfmT*-m#WIw;%Ai{yQGC$F>WjNZC*?T z7c(7>ulrwq%DFQA@P6YoN!ry@>$sY<+9b7l6}N%tFH;~Vr(1db0|D`RXN5&$kKpot+&+q{%!_tw@=CZ z{-BrCyWN3fTmuWp3`?CySSTKCHCmBx9!fe zk**gcIW6N|YQ8J=c5jR#xI-!nck0pfWy?hnu${>3D>0R> zp1HRvV;zjMcACt$hxb7-y>n}@6=->c3iQHCSv8hfAz!(rqhE)*hKLP^*fdnzL@PY0 zpLg1x5*kv}BWbv1FQB&tsd*&k=8>`c;dC3)LL=iaCp zPpefUx8rK6-i;3<8*W@yQZs#Qo$jmJ0CMEUFc|IcwTm&PJQa`mfN0W2O-&TDYlmyRC;9_Cou5Q{eIc}(i|8mBORY)w zmFf3^{nmR~B6XC*5f-tFP)&N7l;F0=(G9*Qhrk1#0LGuykEmbHi3== zmw9Mkf{^I-lck$?^||yzV=cJ!X-nrBt4`z7aunvGUluzZe$Nc~F)@;e+PA@@5$xTG z<*@m+ufS<_fKfT(h;v21G+J)7(z)=YFZ7#csrShoqo~}8u8)9CR;kb^h){oz1%rfT z(ueY5H9H`dzIREg;N@2pBafQuV$|k1sFHtqYgGBm>=Z61KZP_aIigLT*+mp_fwN(% zO0bT;b5{>s_@_w;Ze68YY9cXxAPax*%C|GT0Ketn-Joyj^Y-?~ebGOxhHC_B34T}e zrWR}E+O}Glb(HqriJ-*3{4vhC7vX61$M^E!=+0AdDdr7JTq9(-kyGNtb-MN!gzaos z5V$qQBFkD>xOZlRhmrU6aQIY4>MzU-l>Jzz=w#L`JVr(8xUE@Ix|LUjR%q#lO=xQI zg$qv3hLkc%M_WPk>V)V;RPgD$cAPa}3`jzwdE{U-=FItI85i}+b;37uw`!d5_MgJ* zI!5BowTJjNI>mh`ecnd3SEh?YZSpSd;kZVo@k&lR5}MteRa%9-INLKN)zAlP`chyQ@MMU1 zWVDMx;6OpXV1u9Ip^lEJv=URfZgWBK+AIDznln3HR_tugVv&`SXQ6HKye?5wFDEv< z6dz?)pgXckt@XC0DRT(Kd540WP(}~$F(Z+C-$HDj zrGsp60%q}9QRgF4Co6ZG%7j>6B|1*7l>`)V+$=8cK-}qf0**Gl=%AtN*7YkYL;bF- zz2K_F@4yVqcN6(hrClpc6P7_<{j15{` zq?Sjqj9yxGo2wtI9$4xbcvzp;V;Y4o25N>Iui7%p;1>i}VKc45exrChKEc1~o-~aQ zmGaMjb1B4)c2T*4gf)g{vtB?Rcr%-7rae0ITD=z6w-N=LC;)P;QAMmYuQj#ARE*DV){~9(Ni=yNiu=!l|MC!+qa|sB(hxU(uA@WcAbEr}=|IN~7 zW1z+rbE<{ZL!neeJx^%D&2}a?ELlKpGk18)2{cjtDK{e?+jT!)vi{im%9(&?ItKqm zY_g;~VU_vsMbs9}D-!3=RX>xrN)kUe$`KaKOS_^9t+e;HN^uRD44X8VUp$*_xT~FJzc`*YDENs0n_?xKX&4Z9&Z>Fk3&Q2B)8XyrrA0WqCBe zn6=^2RHAVEc}{kQik*364a(AH_wo4fc)anYIk-YI?cQ{KvJz(f+sKjO8AQ>jFHBBc zaMRzSM$WyASIrd+v_x;!XtfmrzPP!nbl+oi)a!HUhq=bxVh5@9c#pvc8|Fbfi4@z- zXZO87t}MJ1dL&R1oH&)Hn>zUFICh=Zee)&v#`g*8Q#0Czv0VSxdu{o@WlYL` zrd7>X6rB#XUg@o<{yn|#`Q&U!L*-h#j3=exsTSqx>|5O_)BXV;H`{^Vrro7&**5J_ zAybX!yCmE#6I=K0X6P-C5ZpmTvFW;Oe-z_twzWGr3Ko%5@LI5fnSvqf#g(Se0$w>& z%Fv0BjB)iTtG6|6RW62l7cGv_R|=Bb)1xig?n`uDVRO6+EMhE4K4R{o!L{;Rba(+T z*dCfRBVIFXR%@6lIb;~*n8YkRztEcdFhVVY3LmUKuEQY?2x5E4WcEk$iyGB_-Lw`J zu@EctLfbL83JDFa>0xvm-=C+)n|eg`95?MrN95Mco+~e!35eL1E?DF+B|Ml$d*J$6 zsjTKj2i+}hnLU#NN5aBq3GZ5#-;8VR?DAWG=eU&xgfcz)>1DPMoIz#A{Ta!SvB6Y)wBYRfr zKF=I{x&O{z;IIc@0uFoVgcR|37pL87Ex+Ygohqd}>MG1Xr!y*WhQ*umu?t`P^Va;na>x8~Q`iliviGw0!*DEqEbJBRERznt%7&rylk)XBr@`?!_WWH9`$j&`h={&q8|{YHvY-?QePgFhJqbgdCgT+)+5DGdLx@DCNI z5n24^WW8UP9?!mW;e9#SiQ!P>r`^0|!&*0 zZMLc!pjci{$2Pb1p<9t|OgppWjtw?rnxs7JLhpWCJ1xPdq+s)FN0sl)Yp)N@K5XXn zjF5UGjbSdMR`AAO${Tk#SgRg>SLbGB^P|bO+BEU{aaXUX_9hP70|~56-A9n2RSzUK}MdM2=s@6d!w z7sZu+-^rblm!_8=1+j6Zql{dJPUalRu22V;_A6X3ELwqFO%=g?(cP&62=~$R$ty#3 z2kMVJ%tYqV=eQmuQb@mf9%C-aLHj9FrvU5NO=Lyf7cnPPOpn6ozc^)(Hym`Eqaffd zAzp{@#YE+mcRttE-8@GsScVN>*<~-9Xt|}0R4np86u#(QDJYmw$3=^n!Fs>Po=%R2 zsfSdSDb3uz-uRUj+U>iHRD)VqDMfzt@j8{1U?J7i52ue2k-PWqP51DhMlY70#BKOhl9gsa3aPIf!J8fsz?R5ykGH9XgTfFN1($R-?K6A0l z@Y=zJ@%sO?O5lgnaR(l)!Nl0i(idi3FL=8Ve&+i`=$|OaWr7Sd$R=HAkBeXBGv#4G8x#?lS=Z>msGkimBIo87iI@J0?zi0 znmUw;FmS;se_`rL>BC!NB*y16CqL2u^lr2oFZ6}nLRj)q@P(D*V7jBiT_Y8yI|>TJ zdterVgdUbmhvj_9;BBiqz+=9c;c&8)zKYPz<=_i5!YGM=ZezZ3YBD}3e|&@R9MbR{>o;*|1!-B3#U4tA$4tH7^CJ--Q2`YjfG^w$ zLo&*74h0&E3kuG=ksv-ol?8vi7vy)oh2c3?;OW<3MrNZY0C`L*Yqb3ZGi(^t?dC%v zuj}JshY-9jHRN@D{omL1|L%1a+Ab{iBxX^7Ym~*vlfj(OmHsUM6;xro9+ae-xyq?Y zW87Onje04jl*lQy4z&CH?pL%<^bXbdurUsr;gkgXp<6ZW(}Rc&vJZ=3AF`C*`{{XP z$79kOVDa_$kGjg0vC3Rhp_gTPKz&V0b(p&}<-Q&2cD&h+0-k0e1c`O&J|@?05+nfK z4EvXO0AJ6tCm{oja*Dp&x-5NvU~&#*!C>tl){ z!&6`@Pk3EGPL&Du#c_5|Y`i>5WZ&AtzU{UR0fTPYhQpP@jITyu#V`a$1_g(E>$h!2 z?l4lmTD!OV1&95X_X+r}Tg5zy#KpP;UrH3l{Qv6^8a?8oTw9`a?4T~t|6spx2*ScT z(TpnEAQzW*u;`iw0`m<+hz-HWvDT|{@VAmxO6KjY|LQRL(Yd-44;QT*IaJS(E&FfR z&OO5oW*F2F{XCpo;A5IE9Zpf4=5^}23rJYO>B&(c44&&DE;Ja)(vuOM2_ZL(z>IR} zv-hRjJjrv+{>bst7WkI{P+F3R>$0OFuuQk@s%w2Y&3#L5cdhbU(p~O8q3xc5!8M2v z_{f=k_ZR}$S8*8YJ|+N>DI`P;Lx_kQxz(u3s$sSURPSDnfTbg#R!amvZP@M;EJO?) z60c!G%(H^e?Z2;O3ldSRb!4rx3*8Us+cjJ7ke@6M<`x29aKc~{j55_GKfU0a;2hVJ zHY^1v0r?zE?8}Lihu}!TIkUIb&F-Mw8AHZ7Q0H;9Cog?fF;xIH+9B)B_UL26T>($9 zDMF7{L$yYtzs04r&pC})?bxO3FtCnhP~x<|?j-%6S^(+qI^7*AL>=d3x4$#e-q3#J ztc9wv(>K*WY|^c?$cNiPJd~72-g$jo>lz7X>+$wr%(B#R|5Em|G5^;p5O|{Vj)|pX z$DZaSTv#Vw*V$x$(?Ewr=jAUwsk(-yqrNu3Xl${FJLJ3fj7 zBioG29SlSRqeK+T(m9R4pbwtG5`-^zhmZ?udjPrKW~Mc;;KlEsIYv6;xtm9VoGLCu zA~Elez+NbMH;gWgQe=Pn63>ebgKFRm-#IY(3xspcDUZLYQS6V++3cI3*Nu55FRHjj z^QB>jBwp8ogzz_P4z?Ai%d=(IywxF8s?%(4&}z1Kb7|y1+cqx{AY@gH32PLROaWt4 z=@^l%4q7b@jp)un>)cce-#PfJh;Ry-R|L9$9&jz$<9*drYOwX>{4)qz}cIR2$MrxaIPy$^yYuZ~!k%BI3d0-mnMek&Cl#wJ7 zqo_j@e7DOPYzrwx?>d8XYeKU;Ci(LSHyD+P30Q9~gXy6{Xbr1W+n@@bJ{%^G;x9D< zCGju!#v+8;Mpu<8Hq25cZd451KUM{Z8mY|9pVB)mIRw zl0pLRv(g69YcgvFQ?br`sF#cP$H$MEa$O|nGO2@bA#D)%VV3|}yYykLI8Z>6(nxnN zI&*UB`C24OC_O9!Hm^knft+TGMIefzGix4v{ROzd=9NWLDW*2KYh z6AE{)v~2n3KyrK{QNnj8nI|(A5Dv%L|MO%+{fC%=xhImY6%Ns8DtO7B%6Z8yM8@HN z2Pca-{IGe2K|n4`sEEMFO@XqDK~q7`=khgm+mgB*{1o$u=(B3XTfR^mthB#;d(is) zfbpccmv^EkU}-CM4XEI@hh`=)gZKICTR-?NXnAdeSSTK}9!D4E2=RjEKbHeDi?lBL7NWRuB>U!OoycudJ}@q zZ}Gq491j0?&u212K1hbb2^q>f&s(KDqKgOM1arg^IKIWPYqZZ~MUH zcl%~ZNKc%TXGF6yVgpaI=Bv7{@vHRw{p8hbW1*@yyPrR?Z&cKQDg?)6$D9ertHc`L zXkKY|G2P2P=WoQfrnz5K_gST@#jATRd2G}z68#)F!idTjo9nH0|5CkPGm+GCzA`j# zu~8LCdqnzAei;oP$GVQO)lQ>DGj=A6>Qsm15M76%hYzl^MmOYscYI;sM7x;rG3M&g zJl1q_x7uBi%ULEAYg!C&@FK3Q>&#DKS2=(%$WAjDBKybBCSD{)Det;6 za2e5<%2#m_rdct#U-I5rbT(51cAPRVS}v zW3s^QNhHRT2D9V%kK?kOw$1BYW?A*ive>iUo*d-PGa+VmB&;l1$x=mZ*K~;saupLZ z5eS{c+wio@z$P1ci&#g-@N0~p6Q!M3g8w{#s&Zb=$qZZIc@}=>pFPg5*^On9NYRhW zdDEXo_Kt>qhbGx+eCg?d5EcqB@KU$!UcTOuGR~(3+~9~qU_2!a?|@cw!qs=$Lm0#+H6P zZWo6~zxZCIj^m2Z@&Eq)2mu~Ij1iV=Hq*H8X*z- zI1Z{m3c-0GwMzZ#_5U$yHXSV7b_V~X{NZB#Lb$;jWs7#D#=#>{3q(=fq)4Jws^&C` zr?2#&0{S3j3TdU+G1Q%D+VMP#D!vgZN^K#r{gk`Lk2|dZSC|=9?g-VmtX)RJS)NMjT|_ix-~;R|48ZS zuAKsaw~L$HG@ugcH^ZosW-z28m#lgM<4gH)XSBKhVqV3rh6T<`@pOIT(MNO1g-R9H z*Z_@g;gRo3g15z8ez+M>78T~@&O7rfoG1Q@m(&ky45byjJFF8`*1>Aj$90f>&$K;^ zXQrk|lib}ZPx^TYL)aWO8$h5#S@0JYyQ|Zglod?XMarxfGsYiJdlA72#-Z{c{jc02 zLV`XcX)joiUKu&tN&lQ}-VC>)JT-YD{+Opb{W66Qd@d*2k+8C6<26h6^(!{E1X2A&jm-?#^`tud|tFYa&_1CAN~H> zfaH=%+}2TtF#8o^^=N#N<+Jf6G;`!i5ewYJ6L${bsEoV8XEbcXUAF~kP&W!`$A(KTEb`JTNsbaNaSCG)=nA+ykX`Y##T8@cvQ@X zqRIbc?~?chjEV{G*gh`N(0dBs7uQ3#CJ|;eYU+ANgUuPf3%7NVnWtqUC!JNgbcS9R zT~=4ND3j5Ke1ynil5I(T+(Q#=XK(m3LW2uWhWi(%i` zqirWa1YyxE&0?A5FBAZO__ZZ0YThc!@k_;Q3}gqdKxr)3X{5U4OQ@`Ra=2Sa?VPy) zPcdfLjTEmmB2b4_-dxvlZ^~uK^>uJ5eS`on(!1zp9KEf9gj#<$CVNNHQMp5-_dxI4 zy3r<9YQy{*z1IUd61v@qsneQrE|m?^Hvm12*@*os+H!E2_Rqm$dNThMY#5CgF0TwZ z*DNo{c+Irmu`_WC@*u_iCQ{tvU`f)89o>w*;lc|mcspZZ(ReroTZT&Y+C>r`wqWtg zBUiQt>%ch-6@X7W1m4tHH9s5`&N#fNRU_eb&?G%n28HuC_mTsjZktyc-qkw6c84%$ zk!_c_Z3kAJnn*HHtXHj0qdoxq{2;}@%vRCZz3JNXb4-$U3 zCzhIAd1#4Nffwkm&5P8kmyH&(48dQqNB=~?(yu!^0oe?$|KNjc0J9ia8S4kax)LOI zwHs?Ef+^Fqyx2kxg@}9Ya_|Yb8C$T~9}GB0EAblBRWtXbOw1-npIijl*8j!K{uko> zzaY|$Ll&M}bDb+tpCiQ7{FUM2WW8r!Z=wLtD%Sg8pE}MN`3$R74zi(KYy{e!VT8ya zs-fOP1gGPb2ZUCVdp3-J;Qq*mTvaucQq+*OvWL?Kj0Dx;nCA%_s{=F)S3-2L?7zZk zOkC_}U<^!I!l0C-8FL1gn&u-Ys|#NJ-H>Vr*LeyAnX0~i{*g#9Vpmb`WGeOLdz6fW#j2dG)L!`k)*+_8cK; zm3h6#EE4xV#tuUB-$Ou0xJ)!<9)q5Wjn6e+Rn7vv)@6XrZV-Hg;awE?WWV2=d|`wD z8#lr#FM1=tiBTWw&j0F3G!B`2f{CjR2oi+39WLjhz&BGX#_MeT6oc5{-QK8ac&5|5 zOAgc>h@Qyq?ub$0b#j3`nE6&lJi~235{22)Uh)q@ziu;pVqOs0=y4QR9qWJpp!KMF z#83l7zI6(VYm)E>x=Eez(69m3naKD*uPRo@>vxb9{55YfNrkLK&oq=r6Wo4=m`m#k zmeB@uqC^Ja8bnPu`zU$IEqp4uA#$|Fbq;JsP9PEd;A73BPcG)Y=1CnoMjl$%psO?| zhrS2G#C1iinA8#Ikm?d>5@!dhHUP#%k6wEq|GF&sjyITKJ`AN0*6WmzpQy15E^&s7 zDnX?$jDO*lGh~ItsmznErw5BD82Du5u^X?-)R1_4RShZX^&C6tgTuNl?U2d#{_-^8 zR|S9uO4?O)GDx+9eutDZD1R1!-q{WEGgPT(c1h!*4GQj!%-JNN=DxJr!lK-9>+;WesUgQ9}mbuMxR?ktD6fSnhTRB?@ZqcndxLuZjB5^#&U zOm4cq?->%kJZ~5!(GeMrLVy!}rj4ry0Il(j<}&Y}bGJrCqF-KXtP4 zVg>B3H7k_<>Ca#j4)fi=2ui$QIm>4r4sS~i?;*@VHb5HR@MHk$bl<<-MyCUV=_#?o zzbS`sRnk0@H>~fCe*#cgxIw)-m(op$)joND;z?zwgUzWS{Om%lC)Kue7};`f!o8Lw zaO_=Q#-w$C0%muoJ`3XrW-v$JH-Pq#o0})E3eK-I&*L$=uv3TNxpf;hYX$-Bez&2IMOFnoyZ6|}SacA|I z!{Mip12hWxJoxGwcjkAnp8U=XX5*|mRM49mI&Ofba?d7^kNDCklo2VQJ%yR=;6 zyD?UI$@@JL$Y?%sJaQ$JLX@F@xSN+h<+zCE_U!uh_j166%3zvzvygWG`L`0+Dd@}XO*s}vTN z$|6yvd*i~xqW z{ABkHRqxtPip|20itRy_|^=%?~ zoj+~<9D>0$q@DLMQTPE0c_I*TltkqrH@`t} zj@x0Wq;%v&!2AIA7a?x zmJjuDq3(itMhl2TJzNK$EP2t_+=7%sR5)xan53v2rQ4O?GdyoQ5UCG2jQLnt%-iTT z>k+BM@aOwHz!|jI42Nj0C%ikPffT>0NHMJkQW_|zpD4J^Z=*Rg0*FNGkAO)3{_$%6 ze1UP;5reEgV8(JyIaxCaZq#NSLIxLGgsCau@cg^>p%UWfP0C{vvjV*X-EEmewTg!g zERvL0-O#I|ruLyHW_oDf?BsSunv1fg$LItkFi<P8=^~u-nu8g{OzM=Jlmt7Eia%8FbPZf|Z3@*|I3l!Z|^8CFu<~tN zE!jC9SV&-#Fei?_#^2epzNKR^+Z42Gr`C*fGs#*Nhfap;S=3QY-mtP4_0`3=! z>TeGlIQ3?_6O)u*^1g*+nH&g#L&_a0tr=4}`7s2QEH(O%qT6NMnsIQ-6>`F0f_~bB z-c97xFTGq2V=>Fi)#t9sK4ET3cWK3!xsiFWo}2cwC7Hum{8#Oh01l51YX6r5meEuu zRcVY9?~4~ZdE1ZAmf0`ApOwl&LbeC`XsssDyQ~A6Xo@^8ZYh~j9=_h-)c_vjXvHWv+;OKAv<@5BT1Su^?8&VFc1{S2VUsIz6h}!%?xSg9(^lT40uU3!PW3^Rq{T@5zD#*$44iu zdM{lclkhvf$B)et!Hezwlc$!|lS#Qn)Cj%DdBrR3lDFc&3L!XGVKECG%|SHZSvHO+ z5rNqyG;9PLYCccG%peV+Z;92cJWoTy&mZn3){?Vp$WyXx2hHUm&N7)pQvkNOXw#U- z4%*FsN!4vcP)i(3+&t@AdLs8JKflwRT)#hp5Ks0sgtd!#-gNi{mpDdQ7(Vyv^!NxP z#H3I$C^$c`LzuDF2m#-1LypN+L2nEkqCCi~bP=q( z!17z#9u~r|lY+x4b>C#_L0%ZSwF|-ALbvjx;YvCr61f5IoY`;-0BAL2M)B)*2_w`} z`4E`1JP*2!3To^oK1#P!pydM)U86&S5dH2jaM+D=??M`-i{w&w1^OYeb%xt0&$#97 zP4aM4|2^jGLGL3O5CCEP%{hR;@CRwsAf=}8&UXo$o`icVpk|HwXokeWVI{N3g`rbGCzm`5HbLR5#GAL)~j9%Mx%|8)c!SQRm~~;Sn@k z_qMAUlSY!U252u~WP9RUr1 z-BX~zOOCVvrpr8A0~rJy04=TC?oLDHJX1xN%dH^X(YRcDAPUm5^dl)XAa(FrQbJU z`~Dc-&)yYpB(V?13cza^vq;W&Ss%|IG4gtZ^!lD|ogV$xVN83OsRt6Iy?3g<1m5Dc zYBG5(OOQ{4WD{)X*@P1slG`5wxRLvQGBUnoYfCGqPCU=PFNhW|0D56$e(JltSSVCw zx4R-V5_F!?V8p0d0hqzForZGs2E<#b;b0*dp%K~qnQoSrd}xtqenn2?&*w`vpfbCH zD7=yix)ce32(2cx2|zW}&Rpc)_BBSOr1w8x&R*MX#-vP;KqUK+)|2MZ-nvJ0W)r2y zPGppA_u|v7yFcRe?eiHF6CMvlM7khOwUG^L0e8+2*4VKuS*H&cMfL*`uU7zvghotd zNMKg~aO}>971c&~Y1uhz67Ogr2bbdZBjjLu_m~##qBSKbk}_2z92(kip^7_gQ10gq zKQz5YYuE&kfz%??`gcz)nwHIaggSeM_8%`%_6TP3y9B7Jph4yb~h&Vg^&NVSX}8qh;1E)E}47%Nm6qc(EnY3 zKKkxNPa>C+j2BWS0nG29MVu)3qC^0cjeC4#IWCf<3O`@^{zWK*hP3;KW%e(~u2sQw zDUJX5c>7x+kZSsR!iNjAnCD&7_}MgkJ94eEam+dypCQ$M6o&LM6-^yW4*i$?snb2a zYWV#}xkEe4gD6Cc_YKxY`a|dx3;~*DO;LMn8WoSosLC%qxOUm~&Zo_`+bC!)k_h$i zqBB8FrX#6w|6&%}6aCUrx2KQG3ub6wrc%0B=m>jrZUmAQVa^8G!yB^K-FG9{9ogt3q6jUzMOiG|k=Jel;`|f7)3GnV znTvrP_Y;6CqrndYv?;=@1j$u9DMM}QhXw;OgM$`{vu_={M|)d+l`Otgnw2(iA9* z@dB@5-G?nAh~~r{kN+0B#`KVV5#eA;U@d+7bOrY+O;^z41_-%CVJEw#Ue^K}{=@`~ zZ+)hwapqX+?~`5=29-`9e_>(9nsQ1W8_oVrOiK6f6F`C$5_bH!REsr*u=MalcLSEb z@Z*OPg+>ZPCtmgHKF4s+ZpW; zQUAa_rJ@q<&GO3oGvf7t>;^Eqto(do*V4kSrR9)u^0kq9?8>>g(5+$DuAz;uKW{Al zZC7`P!Jw4ePTfm|ks25S`Jbu$QE(@sk&KekgQCZ3XO*HbrUUU@B^dKPUKO z&HEs?TI2ft$`w>QnNtY=9zd|@O1(9wJ3T3%07L9;Jmv0K>$bW;f+6-=j1_j1i`)72 zD0Gp8zj$$z+wRTtvRzH}v^1xW_0Kxh#!CsP*2LfWLW>Yx*7s3@)N9{BE^iy(6$qUZ zJC<|VYeV@O#Un*@n*LHlgDgq50i$b}UL28FGuaOg69%*~5ssFgwcDBm6%#C>raEfu zo{TfkcJ>_I_3e$h_nE7jJrJzF^-wcxtKi<|Et=6X_r9u|s23i=i}U;mz})h|{qo^G zY@aS<78w>}Ski9?bp04?rrP#91+{8zi21bxG#_;n)E3g?4tmY(t8>%bx>&;UZj)xq zJV=(3Zmr>7ikx17ktL?2EQExl;}{OqKY;?MOQh6x^6a=4lX{F;pV1qD*ttB;NSihh zgqKe7pu=AUci05+%8=>=>mF`FDjpFpT7_(>Cw~5pl})y2sJ5E`b9U1^>`>{Wn?e53 zcpTr9;Gc-6RAR}$FW?MzpjYj$)+c115)2j5(rrt18w^x1M6gh?pHu-}$n&Wt?MiDA z_InxOWQ>~%*WmPc-frTs|8bxGqy`AZiCzOEs6=kyk?^~n4y{Y}MSx6CBRvOiI6T?E>flt>#EhXlI>*q)e(%rme z3s>fw1yXCngZ{TZ0k%}1*x)7BYK6~t{7z;*F*Y9|z|t4$D@6D)nj1%XU&1`JIQA2R z2^Ob26ix)l@sgyaYSc4K*!w+YBz&}1IV@Xcq%C=C4rxa>Y@H*tF36#ZD!956=|_+q zp0ik_U1|A^*XOAV9K`G$-hT)EzOz`Y6+Tz6hI4m{9(r%J>Z^R0A#umrg|I;sYSC{J@uIbY~n|5?P} z%{`6ydY#@*H=*3%4&SbS4=d}S-#(5{Ot8*!0t*c%&`-X639EP~8)L1^l-<7Rj%=dU z+sHz&&`@cJ1hS80)e(_jeFDPG$~mFmQ=>ks9WkuOLlSs!p42%YN!{C2F&+)Cm>eb# z5nX@F_(v}H?B25wP+pA%mij?%_7r{a&3NUyrEfmuDTfk7sl=~@R|FF3$JqFGWmr9` zEQqHspfdq*pmTw6(0Cy)4BLz-`qx6YNil;ygX?qzW}u@}K&oTc-Gvm2RD*kj1Nlu* zVl)7AH#;$uCG3v%s!C~aAJ~`| zCXgIWlcAz;JHxU5)V!0SNO0`Df@-FEQ++Db`#ww55+}{%Hrh?Vq?j}Bro2|Wh;bVq z>j%IwZMc9A|LQH;y~Cz5OpNEn&d$S(F6@RQ*0Ol3h#a);8!uY{ed}XKo0l9@qX^4)Prv?OWavb2;eI+J zI8SoVkCGdk)u6T^)C^L@c;&BvXSQ{YP0s2Vx@MQhpBi)QpL~TmYQ~qNuK;lu7Jif! zxD6BOg-!QULj|IpZK3^tMy;5t%bZ2^Pe#x?dCsILv)^^ zeNE=F42$CLdY)^9l&MI+Tw{H_nhS^26F!5f>v#z1X$t)P;iNTE@(7?iPWD8A@P53D zY9#WAJ$+vo@k)12bs@qK62;y(fH_JnNA|A-GN!455bOIyK06Q(*FvW825`mX!^?QL zbLI5lTqGWXr-kXjK-Mos&-QT2(jA|;y0dK|8m`^|$w8+3ckxNc=S_f$)9vmV0aRl=OD6Mn|xSmC3k+`fj303BiY`S!kPZSm( zF&hM2!$ODJ^b!ZtlP`H_x325ZJ%#S~9cwSMFvUc6jiz_p>tQTE=^Ay1*&o91Vs48k ztTM`Yc^~^Ckw(!BAD9_paA!Wm*5s1eg_1ja=iP03k2|`uYKNv-(xIGo*{3vG*5#we z+w5#750>NQ&SY_BEgn$__ht^zmc^caMD3KC^-{{o97?#o3t#>qcc|A2c%>+7VyWKn zAql5hfNs6old9LnJKg4kGB|8Gcb-A9alCVYb5REhavc-@oXz#{Go|77Ib^QyWA6=O85w#nnQpoSW}3!8u~6kjZYHEcgsjS za5z0K(VtFtdKSszxA5qG!ORn|Pl3`0ogtOt4I6K(}%+^+g(f$_dutSKv9iwZ7I3)eGx>6G{^jywv~cg~OC4 zT+~X4%gDmGr>7)dBnIW~9eY%<{)acLxBr79SNVtRA8>+d-M@-N$KaV?l#7&jp?2{Q zyh?lbzofQBYGfH-DAsi_ILO|Sy+r_cQGM8SL~yUeE|(LcshhTL$?-`Jb;vdRM`^I-v=FT8P|KAol4=#(cJHz@FW={=b_|$qbIH1 zt^djuFUp^i1jU+_=95+DLqgzfI3q`<50yjXVpx3RU;_>)0t(>Tm;hOdpw_?m~ zw1?wE00*~k+6K-E!2mQw!mjZP{WlJ|5*{av5a9hZ|8Kn3qwOC@^@9Syb*#7^czOpZHU}QqwXr0`PR4}vZkkWF504FRnb94 zoGOC6&yMRrT38!0A69q+IEV2$&asE}1DMPfSLA?dfXlgLnH9>b&bc1oKKT)b6%KQu z@zDxT5-s)LXMn@d=e8@NRYb-VS!uTAx>Kxa)S=b)bp^)WIMXW+(hK<+Liy26kb%}& zUj&L7qZT`gP5s#8CI@6}o~y{LZU>P!40(MC_+D`uY4}|Y5&B+%>WX`n2uQl77Dm2L z4z9!jR1a4@qXOxFLlz6Efb+_ixBX4DiG#g{j+!>#{9<^tvOuT1iMCXQq_#d!yEB)LqoUv7Ucs2vt;_}Z{ZKle;5NxF^O9a z!Z7Pj7tBy;mk`ng7P*Pv$F)v zyJAL51p#0~6=)TX4$?p1L))5>HKh7V4gstuIG{EoiURp1m}rCyimK3Yd>UZP7fakT zGldPGBLe=}iAjKYbV*?ycL9{26M9?3^HmtVc9al z&`$q`C^5{^pD!5|8yb!Jw>05TCXO@)^6mclXiPLV<%H02)&g!;*zO@GtFPjwxRom! z!pVcHKZu7Gi==#p?jF*h(!!UWk$UgG%V5+5ug}|^KrFA%vO&WJ`8#;794twaepoR| z6a-SmSV09pIPxJ$T$ajDq%s>#B}f#+p+~JzY0(=*&nov`r_`_jI-=#-GUCWEKb0!E zH{3RCJGQ(dnwZzg^2V!e7fG+Zf_ZpacnU7{$Q#H=J^YUmI%sOr+s{{rm7p%57NNEcI#=xQAN6LOT^^*d7=s@r>HQF&c<(uENfH)39%99QBH(x9b+BFZU0EAu zrAMp+VJDfTeFoI4r+#SbgL=eWG&4gRB2=Md-m|cFo)zt!aLh zK3^Vq=tm14<`>W%6~%IXM_Jd~Tnou5iKW`4j_?HQ>*Z1jd1QaWqI9rc)P1KUZKHjK zM!CCw2v~uoW?wch(@;5)bl;#dW~X>EW^S)=J0ni;w`*Fyk?wxg_~#LPR1__;ef}TH z-a0PJplKUcL;(Q_r9%V}X(Xk)yQQTY6a)nck?!u0?vfNF6hvvHOGUaxQt59_yzlpY zp6~bl^Zj>mah-ea&d$!x&dza+&jNbqh(VQB(BDIwE39`)Xr%?v%|f9m;(1pmM z2jU`H$%zAs?I^nw0fW}P&g|6kQd-(HS~Moy7U|cVY4;gelZqkZ1ppQWpnNdkuP;!| zL~X5GOYlnE3~if^?n>iY(L=YHB{SBMD&ZG%!F5?DPLWUCSTz3{&}IAW;XSH+?1R7_ zV%$n{{NScImA5nb2KDu_gnB$Y@wMkid#lmxG)7XLYsSK&gmu(We8XqS%q2sXymlc(WS?oNHtT9; z=4N`XCdc$Sw)Uhi6NAPW!T1i#UdrA0H!U)>KEXHXbfrx~WcpUpG_)M66?*nqsI9s~ zG4MDfD+yH_?WgOcK5Y! zea%dZcOt&bF*EC(#R?`5g$ohNUAlm&UC0yEfs^Sni6{M=xPYsK!PTcDBJ_9`iD9nL+(ZNcDk&q&Jt{|J$O5 zBvp6F4tt2Ml={sfSrG~ALpm3Vdn(6y;)MsYxqr4M-kLIwaf!LOug9kW>>XF6aES*x zq9G0)BenU2*i}SLGs{uUy3vvoby0ieM1^SLtx?RpY{ceo{`R-><}7Qy&9xjfqAJT( zo<7omJ{phWq92U)1*w_IDdYp76Q8T$M&1j!D9n=3YF?+a!GEWf`eqP2t4oDpb8oh7 z^vw>orzkCdnY>REvCiJ=vP8Vn!EF8qO6*VWz?LUui-Sc~2Lqh&x}=VbPOi z5q4hr*UL+ANS8=XY(#825B`AUM>OhC5xy0JL;5r3&V`p{wh@^uY z^ELkxM4N+<1*3l#U@;KDP)V;i%CGq+KOYS|+>h${C%*+MOn_wR1&P~azG7+!cUq8C zrpDugKc|CVV?w-{bVIgvf}hY+bJ5t%n@*|GPfFNy5#@Uoftli zc(!?s0$p1Qewfo+;e#Wi+q8s8%Tmza?=rQbhE!3Mh*c?Kbb`Cp3#&rlgK_g-Ay`+x zgoem3{oMcM0+5cRw1~4WRK2E@aQ)@SQ_kRi)e@Aad<;c<6OtsrUllMo0=Q2^EHbR1 zOnYM#yn&F&CQ|4id!$eHZ|Pt1sX|3EW8k_${~*m*qoOXxV%pO zeyuS#T=P0TDgv)PU~1Hz{rv&2=g5-|!kI8=rQ=X`PF|)aG&bF4!Xs1 zIeN;Uunw6w(o~pE({Y>zUy&T7-yOgbn|)(JP0U7EjTzgGfoaOF0^Ng<5F`n87leu|HW6vD?8q>6a2z zyN-~#K;htE=ZoYFPT9ZTJ)Q54Xv1PDy#HDYVg-!(mwz1LqKEVc4elHNu+qkR zs%v-@_dd~JCq!mb4D+JE=_(ayUW7Y!BCZuD%w)J0&BDK0zEodDP#G{(;v>e{)pYnc zVeiCYZtj);4gp3LGNm!4Obx$;TfSdjPuCiS-x?Cj98(?6Ta?HIgz1H`+d#UiWT1wR zxZGkcg4WK=|5l$!7%D?>hf}cAGDC1@{Dx_7F;@|2%Y7r%5;LYG8Fv+|Z9ctyS!*g4R_Omeb)U;W zZ$cZj8>2($mD4h*j>N%k`iK$zu^+}wu~^5^;&{BG=~`PNo5cFqyf*eZ=O12|w-d&R z@@X~kh-1*vM`TrK6_L({OM~6mUh^XtrNRc48CH=;YW2cP@$aZ8u_a)}^Bc;_uvn(b zSa77QoE+|sLegD4Q#LW7StCyJ0gY{zhE>AOp-Ht}Vg~DXA~F#+9gk>G#jKR8kA+dV)WR z8c!-0q!~1rwg`IPiMobDtwhIbGi6CNSM!ba8;20y#j5S3L)OH4^txk4`R!-Qn9jalh~rbGgkc8L1w3ub?Pa5<2xeZXhD z5sO^~s-89Q2@A35fUQXqHRe_!UOv%N3(N~`$;N}SrJpevVLMBLrSJgf0MXEqco7x- z1be^b#<_~)jyAuU`!vzMBsb(4t;VgU|KpmY_a&eM8N^NKnic9?hPUoX=-<1Csai%i z!^yP}MWl+l1yaxLjA20qBup0H^uGKm{Fw0W&(N=DMn--IfArXF_4(jNSM?sNm=k(R zigH)lNYyKoM^QyWX5Bd8p+Ba>EfDD|wB~obV2Ki;;)51fN-LqLGa+=`@YCyyh1-wn zF{pD!Ex|uu!Yihs+7M%qH_v4mhMIzx#BHAf-j`bS{KqR*Rwn0Xr}3>XOn#G3Ux-^* z(q`Lx2PMKSKeV9=?P8DE2GbA$spcY7jv(({t@$`2@E}57vC8yZNksK-E&k+0S(Nrr zpHH_TO|ik0S2{UmZXk_!m?^`mhb@$D3jHZ1)PmDsRHISa#e9u;3J^dLDJ&}+!$DKUIK&!xJDtRysyt?S-tz73620@?9#pfzPfzDR7ai^{2NK41+!Wt_ zf{$_)Wi)7ny$p?}026%@5$z=(KImxq6*HF0K3K>SdYq?r7hU8-_$y&PUCR6u`}$i$ zrUne;ff#*xb*uEWy!N+XWrWHwo47oBy#0xdFU|J@H3|zg7u=@KeMYuTnDjTG0H?{$ zlIUhW@QKhSnjLhz9~E$lfpi$ww+eiz*-fgQ_itt?p?(e{QBQi`JVd)#4|WmJfhQ zuWk>WF6GUmqml-`Lbs_qRv=U)=d@CpH%mzq$@IaHXehufBzuR_MT!hO4H2S_zq&s9 zCxj&XUew(y;8St?B;XeMwmI=`^S)?#yBf`p{thn*qL(RrF2!6!fdA1<<+d+%B^IID zLe0gl4$qg$?LV&Y+d=$0D<))f(Y(EI4aNEnbTT1@`VDk)6jgs|?^#0-D`#TJ*R!XI zHI@D%Dd;aUESJ(VASwje@M_n=)@l)AwYr@6%}rFIJig;-~K~_Ki|S5x^&% zBH3;rMB}9S$)F~0d0VuRMCyjf2i&N^;i1_ipygGRfMZ7vACL#BlS%}LWS|3A5nfT= zDK#{Xv_~2YlSVUo&CTb3p6!3i{DfPB+Rst_%^vKEbTc8>eaUa4uUox){2vojB7e*7 zPSi^j;9ajV2+Qw3RYtvZ(Oitk#BPp<4xP~i9DgsP$AQBB+lr$wkfHQ0CkNZ)<6_*5 zS`{=i{*=sNIp#t8t04t1MnnmtlR6}RsX(f*e>L*82{KzUm%#?y(!kg86S;zTG9Tpb z$ir-HqPF?j@Ez3d()w@QOoFV3WXHy5ENeD${W5Ddg?MPND*Z403= zLREp-72Vx9#BCI5ghvbBpLGhN>V`mj^3ltU@NG8gBM%1Ern?!cC~zOA3TWkEjYcR{ zsC}yj&}JnPxrDG^Q>l=FAg2D}{qsL{1K|0dC(FgSGW_s;%{bK|1TVS~QHT2b;ugyM z1iXZ{Du9Q9izQBB`A$h~Pa-A~r-6tC&uX2+cik=`Olq}I3fR30)1k3VEZKjDi#25s z)md)?HENXztT@bC84_5B7Kp$fIM~?O5C&es?WDOpushNjO+!P_46FoFxe0i~AK5^| zszV=A!w_w-k<`U*)Wh<=1>l5noJBQSAl@KM3B6cLTT{5?+O6CPa^@ zzGe!#PlKF&MLa=JDfVd zUqAz2AkF&10m~GENmy@BE)k)!g-_L5IdR}vvcbp0J0h9vKA?idz-pzUH{dSTE9GdT0mK{hmU3KXHz0afeTp>;twi-*>A1z1Ju4A_ zp#YS6vITeKR>mvdtiieTmO3&PR2T~#dSdRsb{;@G!CAD%=w>Oc0Ro?{Sh^B8NJ%m5{>MmBPnw_t&80j{sZLAozY~5J2vI43u}F++9g&M+ zdmIi5FJidoy9-t%QO$~Rl6Pq2h=HijW;kE*UvK`;PIR%Z-rj@_l!6{ZjEV?aa8~WR zg^zN07k8||Qo=>Z`=%SyJ~*PIkfT;6iFmuDaK|-z16EFE3u@aGJ0X9Wqrnrd3n2$7 zLC^RYnJ&B*++w1)kHKZ>B)-h|honR$WM1KMn;(V^Y(!%(An2e8+5S~i#{af4d!37I z46$baSCie4nfjtz(2EOa$IAZZUS;N-Vhr3)1|MX*g$xay(D4?}mt2oOTq>cs1}fKs zi*DOPtoohZrA_lf(jTBhQx}|{nY=L9O9~Icl$iN_h@)4J6QGL6L?2x&e8R*CqAz<) zf>4Kn$tDu`#0{aN$Ea<;EdAE5l`vJoZUr5?3X^vAmK}V$0b-(fYz#-#+|UmIK8Szz zyXPtdGc+D1TtzX?)~fV?@T?kyX|XBc#vRv61jYCdrk57F=FVHlNCu zWLb?qj1i@vz_^{fm!&k8PtZiR@2Jp1j~G7UVT091^MUL2Iuj}L2a?GsewTb|Zld3; zdTqWA5OM_7@SiT-M1S83U259tya?oxgEI9$T~vRDK7y>jYcC}HVRu6Ol=*Qy-qLW9 zq#yzX5>ZAOUwic2K#>W-e$Vs__qzqiuCpTr>f^|zm)_$`ik=%TGO@+?h)|-G3cK!3loCuV|DP9p>v8(XKG%5#dMz-* zZtf>?0uwQBDHj5*UcpOZHxG#k%yj}XCvt)aF24VxzSY66%q9IUqpFF*zzIer?p-J? zWO|9YAQys{d}N>*8sj!}m<>iBkU8V(LgGKAEse%dNMaL3K({H95;ly~SX5D@prJ)Y z4DA16Czs)`%fxmN0*MCWBQ52B2mE^}WUG%WC@P{4@1P#wz03A=PJ+wx=$UO7UQt;P z+bfY`lo()E=;+<71qm+m2@QBbY48?_E5wiryUUTBDn{l#o3 zQk7!QpDDYH#sGsYtR=*nuCC=h!IcSvnK7}F1#Pjqh4O5NEbm^Yv<3?M?b=>>d(0pW zZn3mLkt2++0eP)A=r1F#uK?_t{ibE=7U&~jiW*%5a>3qlKJ=+nYWa^fgNESNL9ag7i zk3>bQlA5{zsZsEom&7*Nd- zhy~>MjuuhGTzDDCZ81>b%0dmUE~*Vl|I-)5$;mH{Z398zb|O>%O@r&IKU|mM7Mpfl zTqe=rqeG2>R=AH2PqN>Mqd}>gDU8ep7F7fRamL~edQ6V&^rE!t63B2!kl}ok%ag$^ zL>3I6$PzxNMGt;u@GA(->j}HH9;NX*kAO#5O5EtWS?s>QW?tKly#D_+!w`flvhK$C zAC7|0HE8?$t9Y>lEFVaSlSSB5v`EwWpKEy@?Y+I&Y|J8IKnH7a_!pM%|8vFq!g@bE8)PxEZ zNpT9{#e1M4A&LS-(v^=tqU=L9qYmqExdA`dlb<60HqQgF5|*tI^k60~`7j}RYf`3m z5B)B`}p3GpSd;uc_NpAi*F+K&m;_e5~N5JDZxzCpq+0R*7kHi58eqmano zVX>p!MMI3B=+$bmaaU0GVN)1S7oW6 z7EG$G%Iah%ywBSqetuS^VL>miwMJ_rXj#;2eZu_1?dPZmA3XsPzEx{?d$4y$Rmab! zSo;cdzelW%XV(SR7|ph}7yf&S#E5ctS0|@FFu%9P2>wy;5c($XbVw73U|ya_ooW9C zKKs#=mffxP!T|!+H!p{dPGrs*PbiD`nZ#VmP@QkJ?uEBhU}C;5bUb_Q$mhFF8~)R4 zWbbaq8w)lsYxk<#Tzv_0$~(6;M`8q~x>E!878*Egnq^Gs4!jCA&X9jCp1Yd8S36tK z9}(bocUL16jx=5@%kSM+QZ7#%JB~!7J@ofSo2oE5UtROH`(v82^=8OOY2+aOpd?Oz zyJd#EbFgwIa^3Mk`QBX$c=qrn|G2;ye>3Ndca6~wCMFIttf#_nhP?b|oV}*0w2aTb z)!GL!3d(*NFnx6Psh=xyUn%lH-X<7QbS`W!qkQH{c+1Lo`L^D{>-<_iZS4Z-7tCsNu=S?$sV3<2#9)X3Zsv5pKwEcE6^wcoK~A^0Vgb z$;n9yPgBB4*D6Q(GntC3J9J!5NAdCAtX^(;?uw>Np}ng>c)E3cF)}S9j5OBoWRD?E z*v1`S@Yl?9Rd<4&FQ-Ij$MoL@?ws^)$-YtEJ$;U*?oAwtdmubjA$b;YP_nZtf{23g zaTkwO*1lw6&_}!nIf0VaRO(`AseM|RRty@oCS`Ti~u6RpU zV($!Tng6kF?cR+&!~bKneJXvtIOE)PA_?ah+k5KFo;j@k&phQVuiWGUae2oQYo{>B zXDkTYH0UuzK3!&&H&3WC%QMU#b57_^^Pi!n8OkMWzXCLPLaJNag+hvI;rxA4$`?PA zC|j4RNGv?;QA@7=t(SWmql*PIgLx>?{3YjzQ#aJM5A7N7CVX~G_(Hmz&7>%7b4J)4U3>1;dnL#_E2?#IZITBkf#0T?$-~rc z;7wCr!wZ17Nw{8vb_CE8Z_qJ|*%{f5`hNs5pBY)P& zxzzr97&d*v&hDhtqRg@6@#3G>Rqw;+%Ix02r!bq+rQz$im(y~H*|N%HPyG5=b@*nb zSIWn~PEF5*Z7!r=uR$K?6o(n}WoRdU+Hr61(_85>qSewvY#%my?H>sW$~z0^u5CMX z%h?GHgTt{^f89zoZky12ti4!L<12r_eP=83Wf7U;AN@HGkc>CN# zqgCoyv*9$N`|CM#cjWN$*13u=bRzcM|Lz2j%`xu&pYLj`gI-;%YyyyWySqIg)qK19 z*||h>t6o*>t>-WA?{ALR+9e)s{cI^=bp)GWc5RmHaK09y>VsUcgQU2s)hL|uV%l~-pD z2?g|NtbGehE>^k@M?^CKXiOSLBN=2U$Y}zp`>^@xe-)3QlLxv;ot9*|^wbCHqq4@Z z8-*dTa>k4NeAqlLNG__=A0jDY@ZyVt4ohai7bI8eMa~=+#eDQwutVWND(Oo}1h4&n z%30LiykI6l908=iX-rQM7r=tS#o@DpCY?X*z3d3*X@$)H0$sb#{jdJU(q0htE9CX} zB^9mZvcWK~v;rr){-n?E{ax;0b(ufb0T zxBt)nDkVHb$m}`d-PL&J-_KxaZ+QuRtJIxOEMYx)|EaAt8AS%<1JMo3kk~ItyV0BtmLo8jj@^)VX&gin>)# zlhx;;hx6EK$H$-p#`RYGZn&lxNfjY)Y53t*{%3#gvBa_FmhL>{Hi(R2e40(I?k`N( zxuY>I#%lT0wwa{FP3Aha5EOwbxx#ZUesRc#+!y6ct(i2mSGCZa?)C~lm`arWdnzT4 z<95ZGv^@Q!d&ZS%WsArWLd)$0ogP+dvZ8vN};U>=E>rNpvYc!L8Unr;Jtj zCw@Lk*PvrWHnio6QVR;THd$HdkGHAK?R6y%RM<)YzFqpvsL0M+jC{@RB36v3m(6zH z*J?QC!*J&ShV>(Ckus(`Jfg07TRL~C038-kCa;f;r}J)H)yrI6sT!lnQ$-ba_;wLP zV?9+*XnDU!v=%09OIrnZUS+S2!^@#L&@YN5=#nwS(2jEQ=VGWvI^=|M>V zjP-{X?Zq_`kB^OqbMx-R(D=pI^5dlA5TmV9f}W(g+@2q&Yl+lWz$m&yJg|B1IDfUmn=yZbT=> z;XT>!qh~Z^nshOjI3~b!#Sf1D0KwmivzhZ_?|U5-&%dc6bYl=z2ZfYBWA|!N<=~3kea#p@hB?t6(M@!#cVcZ&thjCpp^tGE? zCe5YpyoPFW_vc!wY11>77_q{BNv<#IGUKUlpE=YxWjnK3=7CdkoQW;w5=A*}O5lXym)6s%&P7f9r=O9a%g#HSE z$clA=V2oz6kr?4du>q7grz%nbz-8lY=rxpgx$2>tfYjAo?oU_5*R!vUN9gTnXzPJY zzL;mm1A$R#1y5|ESq}qy*gcn@K4`^Aom1i78FED4ykhG2iy3LADo`4;!;vHt1iDME zhrn+EX}z4iTjdP7yTyQ(5O-M}k~Ba-$Ne)fd22vx$o&O@%pxfE;$+2DA3*F#QaJZM zu<<9*Qh~Y(-kNYslT*Z>@dkW<{7LK+1fCC2n9;g0bAx{)6OQXP?>Uz#K<6w1a~}|(Og#XK)Zh;= z9XGrBd8StO`UK8rZaWHm7S`Y1!ai>G7XIYc20=-x%i1&KVX2As+EO1~0}dFc zD$F3H@dFr(4ac4bpCea(1=7RZ@=VxXhM`ylJp_A$Ri}Vh;$k!?ky5EQyE;{;qo&gvFRom-viO?m=Q;6`tu(RPtE|KZ139g%?C)k8*!I`Cy_Bp;f+x9*Dr2> zOUL{YJMj?8U0}Ms159@)%QmnxxB-Wv?yQG(>R666&KAHO=tJhWtc*vd0e{c%c{IDC z0PV-$B}s2U0`H3noU#w63VVAEx_K{ua~@!xu)aJgyn%2#_|G5ZiO#-wB0e=gv@>y# zCs@qg>9_&N^gp>}CC}240$*gpULKS3=vPkhrvzBzLn~V4cJ@LIlr!q(+JSHU*Z$T_ z>zoTfl*}!S)ZDQTLNQ?Aa}yhraEhNsf7&AQVbS7NR>Krpo*Z?p+1>jV@<+yX{$bV8 zx+v}%hFKp+9fQ>MB8I}W@rxx_=RSQ)g)pfq3F1m=dTH{3N$terNN(_V!- z_A@dNfjmb~<@T)dU%!co_yOqQ_0^*Arf}}5&e;C=nbjehsSuW^Yr?m06$3?ry|Qw% z>h_HEZBGr2mUpgI_cSnx1WEgcjWFHZZKB?>MJsiqzx%rsN5EXS(FmKt^)D^J3_viI zt(qf6K(am%xomC;BnzK`Axo&QUz6`Q5QMrRF#ZDRD@4I>)hfK+-#>kQ z7wE*@fxs+-(!j7Y5G!ub$kQFl_`a?O`U0W|`8Nwd48hu+9aQZ8%=x2Kb2KttL%MMrE*Z+ zIaz1{XdgCK2fz8xQ=3H+Kgb5i?C8xW+kK-(w5<>Dig)$?^wU0XAd$ZvmyO;{EjGP@ zTby3PJc>6&^;qM|yBL)+VGbq%1t87hl9WZ0q&n3x8RT_clm3Y?x~-XvehzhdKr};6 z+srie4g~6SesyIvPw`qZcEJ5s{B zP#34-2~$G2LuF*&9rW{YRP+c0n;&qy8~FXC-vT3Td8a&M?9*}M0;Z~Ub#8Ma`x<>t zO862pLGgQ+^`-b0fhf#t>%A@#E~)|EKL*uf2zHu*)jE-*Xe}vdPN>A_WR55!<1FWX zx@_z{H}|8+wpvGk%VqDjO`Qv#0CarsZu4W@7HW}WZr0Wlr`eh&)a}R~s)EP3q9?$a z!ICD7faD9*s}1b=HSC-2Ki2nfj@zrGhoGE6uWvIh3JZ_vs&5Q>?^C~_XXbCd?x61J z>;8}=GFa%6;dOfgq+Z%7=5nMOZd5VedBZO{$vn@$L150lzvm3nV==MU5wb7*`dczq7`VN(>YvpF1SB6OS^z!g@6mbL z?LJbRx(N_t$;x9$WdsZBBQ(gWpO^KZKL`FgN2!q22Hn$PNpD7sQKZrheTBf+uH|PR zsjQE2?sxiwczMAEO~$j0Pq%vtKINs0ob$0W^gAk>=?&~Uo>XN-j>JZouu0hS z1MVxYY-cxX*_`_@s;Knt`H@*|v735MA8Egavx+MW_~su7!G`ilMx1y= zeCm-nOW`sFm-+AM$GrE(e>beUkS;_eD@kTN$h{${z+itnd~o3WW_g?TyP&cYhuj~h zoIQsqho8;?yv99%i%_yz|5>a)Ve)^>e|K6{oDFbn8aC&x%^W_7IfVyLXfKOVqQHfi;oZXp^B4p-9eY~SWIB5n2fh09F%+LsYByc5gJ@6-Q zCQ#|2>>e2JCB-m+0yTicV%vv_^4NWG(b@zg@kz~`)lAlc{i zdWEHD{4>2I1efjIE8tW8uxHk|Vqil*oGb3u>yOVuGY<25MwQJUB(C%6%+e`Hxl1-` z&JeV9?ZkemJt%UtQ%Ya^wY0n694}of;9Q2~M!CSJoyBmMrov`j-25x|meaNkSF8-= z!y57ND)a@|bXi}sc20>6u-X4kI!x&@qFT!|=CTRGIoKT6m-56B&n(>PlAlb$a-jtN z+yXKQZYTzK^uV91OMce*)seV7vbmjpYl`A34opM;^ytTAJG1|3lu&(Y+wMy1RNffB z6LIm&0%rG?yUo8dN62$X#rMZ>0W> z2`g}OcyAwJTj?&oL$8!BfTzIn&1Oowdax-zYwAZ0W5grpln16#g5HWfG@8mH#@cOl z_RGff9x_7>=6(*V5C=BGqhT+$^+Tq&zA(qG+f`lDRuk@?&X`vb!X*hYVL< zXz$w)?^B2pwv_~@(O0DDjBf@%yb$x^-rCp(| zyj@B5l=k=D!%<;;&uP4+=h5wZBymeXL~!&pcKFunQqiwjxAskr)A1*^Z=ZL|pYPuY zojlD=)OT%_{}Mp2SJ}UQc~yJ(O4cB&M+S$Z*1qk%lBbcMc@Dl?NO24A_B4DkOQ|O< z@GDaPPLV<8APpCJ$r6j>hv=f8%d99$vP`d7siGhS^6W|t( zclTdQu)f%h@p7CiY-N*>CFXCA7v#^At{rdJ{xwMd2Y0{@e3uZXQ~aUw{>fkyRd%sU3=w|Q|oR(up6%- z=<_7Xq%~a&=YYYWVCVZ-q5Ai4n=S#nZt3jV=WC_4=H#Oq+V5Ex2ng1?n!Zg9?yli* zexKOU>{ZymGFy9#hJ&*|;FiMzIArTy4uWE-y*u1KPjHDVC`UELx<^&}#y{RT5Hoan|JZxh& z^q1?-ih1(CZkN3*Xl$5X-&=PhEzqyR@n+p#TO$CG$5@!GJD3~yp6WyC931T%H6qczTYDQ)0_{McHq)-w%9AtZ=UVzVB;(XK%laVuq#2~L{ zM7R`)iG&maSBIMP1BZ4=Yf=rd<{7$QQMltKi;5ibkUewB=+2*c)dVbU{LlxieBJx( zTgt{#O#YOkYd0DI>o;M5 z+yJrG!Pnu*HQect+VoanZSDAkMltTlP2SGH+Ltsm7Xfs9BzJ|$pV#~wj_<{AHVOpT zH2$dD4p&|*pH@sC5S_Nutwd{Z4Ec%mjlzAv{Y`y=1WiSx1rq^{IGIa z(wjT8ez)y~dDe5|AL6#eQ{va6n*6f(BIWjvO>jit&Pyh~Q~$2Nw;w3(bzzpuHS>7- z^!oPl>PSx*U0dg_@)Tr2GOS70^e9%dYl|V9Tq5K76VcqSnbS>$QT?0wTQGmd%_+ix zPx-{fYnx?is}Sb$(m!-rcO<^-n~Yejc7Cv-FV!|gFRna~`SLC%timF!KQ;KKdr4|R zsvAp>3_0nK_&0_hbkoJfUH%GCiNAEdzKa#vAyB+K7aw5w!UFvWh^-7*wSQDgSb5-J zk1koWG9xeO0FKI4YRf&enNWoEqA4_N22yU24xryuMiMD2XzbMGJ_^ z(ockjc~hF{Nq9%^%MwlT8ka79RbdvSvrJx!`_s!xQd_b(Sz6yX2o-yo zE0cz`?%8tVHSfvXiPJ3g z4x!izSXWtwz6W-Pn)+fqE~M=B6J$nz=FAZgVB`5~?9um=uX#-}&IzwNuryCtIwm#f zY`xuoS3jY2XPik(F;%25Pjur7tP&?-;}q=Un?*{EdD%1i=AW*;!@8T$t5O!9x2yK> z>-e*GY+aX($B*ZTT8Sxl-t6NR_~$HWmR%(^&#XsdNr!;gpV5J?msTAr246UsglgQ0 zZE9P4-7dzh42m%d}u998_%Q)6Y!Q{G%mSyghQ6HF-6zn!n;F2s3>Gf=r|*W5$% zhx-HTfQ=^~3NE`kuU#mId-UHtR!{mrtQ%hf*fj#T1yhZdikq%K&35-c)h;!hrB)RF z`w_>Z)*FIwqsPp(f%}nl*vHhnjaA(0=c8$6A8PBcnk#`4e*MU+FU>}&j8Hv{q?mxQ zj<-v%ulsSvw1vjdnSgkMiQhCY*w_vG*0#{jq?|{kWliX*J?#3e^=k#Xo2=Ct-&dB{ zdxk^Zp0@{`mAJ^uDNlPk9*wujdj5GovrpIE@X2h!g@T2coYbP-OO8WgSFO;p{Y-h& zX55Nf<9W{e-E#Y56U2iir#UUcr85q~j8~12oOS2V`Za>toE3AVP*SkZ5#z|MVf%(K zqCT<2iBqlBur`Zt(5B>8$|L)t{0NGaWgGcDwC8iRp#L_&G2bZS6*onXykg47*tSEmGI@2 zB1c_+#VGfl6}QYzBj%#HRlj^07I6%Afl{E`q9?cV+bXaRl?Fi3K(;obACUxmB`WuR zxuyrZ;K=@!a+8sFIpP&J$pKOfcUv3YzvfBSsm z3ufI*al^LRjaV)BGl>6`tZQIv?5YvRtPV=w-lu-x>%ikF4Zfe}lMd(o*PO8UZ(ha(L&-fV??b@xIfm&Q>?J&p{! zgXHoo|I)yh+2^Fcm?sPmA86En!6*t7;lYtZN#vQ(CXRHyc2d@#C%VLRxp|g5(%Pj4 zqRQqytxZMaDRp)fRdqX+|E5S&->yg8@hFf%JUL+P$@O{@=QU%kYlMpXL+t_a4=f6` z(rc1J7IwpqNmWu4eH0Z(cyoE>9~TDZF@%j2d)7C)^~aPPtbJ#<8ATiP=Bj(M7u)n! z?8>wIS*J^H?@>I5fa54m{m?G^!<4g=vHBLxL>ysAKqoYm8unwp;o7V6U2#S=KTFrQ zb7>@8+J$&%0V_U`d+mFC&$2Qplj>WL>@ zBkH(+Cng#uh3s6Ef>_@l!K%2JSf{lL9Kk)6*37)qrJ7N!S6`o)?-bdswgyPcBa^7a z&bCIWorD@HEBrw3giC?DM{PEFL4^8{KwJO8t)h7I!e1KCd2TT*%V&TK_-KFn2y>hZ zT^R?vGNr-H$@p&^1XctE@@A_)hXUNqHN}VQ0H+LV^!loqr|%S)NN9j|}-KKj)0etI;+ua9PsC3l4m+UnYzEs#0beLeH$UBo_9b}qAW5jj*>5o2>ro)|65Ao0uINI(|+_lW8fVaF>iDg=)%<@u6 zK^SQty+z@TMFOR}`kYnMFS3j?nop9?N)q)fGwt?_I_jedt$5On=aX4IW-+mesvdPX zQNQ(nol4Zm*rW2j=$FXPX9-6`aa{F+&Im7yJs0y9aUepS_EkGf|?n+gT#XZw7uN53jtqu+ju2}jfHXA=mc zsJCYnul1LS{>Z=8$Y+$L8W!qLjG zLji~hl0~eF_VU>(ez#TH`59~JoYc*xQSyx;u&yt(a6&vVqTaOdhH1;B$s?^ZxW##E zI3p1GAl8)M65S6+UX!#)b9!>L&pF^xt#-P>g-fD6al7+=Qes;anKAh+_*II{V{@+| zE8MY@o4j%p-n?)6K->k)Xaz}eYB+822~a{AZPD1NN-NUUB|kJFW+ArvU{-^ET+a66 z*~%fyA)B4{N<0DSYvzrh#fpw_glONWu1cV|Hs4PW$4buInGZ;Z*S~Cb)OiW-?2H@j z&c*GlsO!WR7*Z8vthBnn_g`%daeRX~-Q_nGo+kws8z{~b^IJ><*XsC<UJi(meRH~DxqiHl z@^12@)H=b0yC|O#=8XVri1;kneU3BGn`XcowK094Z_&j~=igB^KZ`OrEC0M~8#f>b zW4F?UDn+ok{b@!h@W#(L}xET1s)b;4{X&F}qUCfHf&f z53<)H6K}Eaa4%$OWegD)1%4WfMcc0%|Iu%AsWeI3{qb-d1IhITpUnC?IF(c+CG%;de#%LBeH;)5fDa95yRy zY@9(y1{@ZJr;BBUEx#eYRi>Og&dVO)*OmX=_K7_`=MTRJs+%uE2?hw zqYS0s>7J-_Jaw^;pIOV<<;RFMIm8KMhIpG*UVnE`RwIX@C$qsP-h-0G-ZB7kY5?e&TCIl-ZNGJ2R}Y-){?#z z{EVD|vB*U0!h}FGD^84_$8W`JSvSP@v%eWut|S?v>z(cy7{A-2ex*U_L+#@1mX}t| zOVkx~ww_)W_jGvA@0pq5b)zbqrol8&jj|@Dg*Wly*KFk>wEcDa%MND!P`lG#Yo(M~ zvk2)dkH0#GAK2@-3lPd_x(w5yI?LrV>@GYRbtKb=ah!>^OI)i*v_e5c$jr2*#O!}{J9?Wc2kpu_$gu=L~! zovFV`v;S~dyXbqZCpThuDt&UBh}y0;+*)dD3Lfp!Rbb&a!U>)t>RvHcQ~$i6N|4Wv zHQH^o{n5^*y1US4OB9VlW4Qf|pqby(b}<@}6;*rlW*`wI!xAF>w!Z1M%V`&DIX#2g zJ=909${Y%lI$t57?#Wd-J~IDXGF?cXj)zQjb0aAu@p(d1til_cOGO%Jj?!?SiFoNt3~gvVrs0w`<8>+ic!0e zs(#fOCRV$3=~V2|Zt)F!)yL^MEswGcO3zt`Rm#?T3Yj8BUuJyW-}87fVH`NRZomDW zw&L!VFbM@GNAJXnK=UofgADSAzUPwo_vLwrbUyQ@YwLs^HJ=YX`t-|<aHZ`X5G|JI2xi?Fb*b(5X%q7|ZTx+bBwcG(`!F+uE=*Z7z{Wstf5 zmGf5q1JQ%`$X|H<}-T5(VRb%CdO&KIm+nBn}!@`?WtU>mLst7Qexp zYixJguXE+it(*GhH;&!YOT2V!eXXQOiyoqwuP)8WJ#2C&&6=|mevFTrjao(jkR>&P zBwjpI7{d(bCVXCCkGz+tc$@K-*x2fE6Nlr2L`6Z@Nh)@p^9|8%a~u~3s2|-UXn4@y zEC0=0J=5Ue$sD_K>tu_*#me*I*tJ*^0td6L6KFjRB)#tYBD9I|^AD%Pl9Sk*oSHB+LBWBTHcWIS9C$uN*kVTKI-TaQi z2^YI*OHUZKZ@Hxl!A(I+Gu9cjuF9MjC@V`3)rmVuuq`oGGkUE%ig0?v-Wysjjuzbf zUe;8F8_x1lc4PlZ$y48MYs9~}c7%MxX>Es6ybZ@%g(sR4K2cO-R(MXz_#9MihNuCcm8mwR-^JXGbz zvdukh*JsX6vj0C@*AcL;2cXA$JJpVFHXBeK@YichmY{&lF2UsVJUkJeFN6O&0_lyK zH$`|7<;lfe8a!gVAr-Y~DVH7c<)~4=_H76F{gcBl)LUoskSF6ZnVtR%vT0mHdK!ST zztK)qGLbpy;kS}Rv%|r@Z43X4pF>t)@x50LEQt>T)|?LS#q;x+s`bg5gJ>ksK~mTl zIS*;>BH5f8ikH5V)7|)=f9kV5#Ts4QQDk`T@Hsanwi`MK5G9^3WcX4~rDQIWOjAIW zA?Nge{Q3X1_vPVKuWj3rM8+nBNGc?Ag^Yy;MJmCN@gDEqq{=&gI$U>k*xc=da|K-^BHQ?TJ-B0v3@TP~m+R44qtC-;EAf#Pu>?H)M? z&trTMjwBnj-%GzSKqu7Xryot9>%{X08DGye6FSx!rVQ-Xi%-$jdI4AK9D(WE3JYh_ z|I?2{d#aSwdB43i9oD;&Ao?D0cFEv*3fO+y)q(P#e*70l=pQ>*^}LL5qW<1S{$po+ z_WpXl|Fcp5?>8z?B?L|4M+ze@DYGe7g>^h1o6b!YPTQx3o_z4-%|Kzn0Me$xtu{JN z!IP z%Xzw-{Q2f>^&up{!IqjfkzSVvB(XLm(jrs4So;Zr_XaFTy!Mt9L1}zhK82IM!0a^EODy+TglORTbt)-2uDSv$(_DzdPEWzG_Y_HF zV1>`1tH^Ljdb0wi?X)6`bO#;e=S0K{n(KRDCL-LA+@viRwBb6p4vqXSzqB%HG0MC^ zuOg(t)|`4-Mcgz88is}n zy9^R@f%aKfj=#0KF(0{UPQh33L8_6u7qD~;5Ogpw_|+m)r5?&rj%G+*feA$2d@Jxp z%Y}`yJ0HTfacUBF6v~WX^3! zYIC*Gvf3ClT1>J*8v7(e&i)ixPp+{uk*lBe$L4_M_R$t7tUdHaGdrw+!*S+7ciBhK zKgmIP3Ki-9*jk0Zs#qN}Wx-b}-%A#LVxKmtg(FF{2ukHYw1%7qiAA?8aC z&3SL}mA2R0@a1v>MwD z!_l!jC1;3q2mHN8*ZkWTK{j;Idt`0~U1ThbN*>eYS1zXVR0sgW;qhz-$H21})}ioH zA{RC*CYAQXdx$*v{Yr0m+D4&UZhB{{2YYwx*wULL+vk9UJynN@-H+MRiaz^9sj$@L zPvs@jENZ>F5$^+X>8*|ugD+(4IBD$#WESkP_=V@e%=Nf)fnUj^EECia@lB8+50^n;-loy$V8%(1N(Z@};anIWPvUFfQhHh*ESww^ z6Me{@XOlNMQaQcZ><^58vJHPbOz^i;d#c2F!&5AJ7kYZGpP#;xC7{xmb^4adgLiA0 z_uk%A)59aFTU6O zs{;0dn$EHzEfOR0-eTGyTX^YcW*VPvj@iA=@& zTLpqgg_Mm!b3=0BytNPA`IA@2*nLDw`3&L*1Y|P%FLA3~2u6xG)-(?eOzeLwTT|-5 z*;5U?Y+4UoBSXSYHmwUNav|mvqx{XkP(6ah_kqQS0P6q>uqbKa;lG2CJ_rvF207qQ z26^soFO1S2Z@rHYUP$#0$>qw&w-TlK3FJdcPhz#km0jWpAB=cB#>Lt%6D1ko0NUjFfTMk(3Tk2nh$u841kiiX#m)vsW|4Y^ z)oF%=Z(xI#kPQmE`6|5lIT9ujRhodV~qm8;0vv>;L-y*Nm-e_NRy+<1?98buxh-zThfb+mfCWb0P|TezvT)c zf)(J|Ofeu{8y|#t^cGNoda+pqC5{{col0x=ub`MVQRe#j^(n&+ri7tav5y;Ia2!v1 z^kCP_1)5IyoXL<{poIziYn#m7fQY7bSZ-vT1hoDhNRY-eF-s$RXqrh zOYW-$!`y=`JgEJBU&2~x?$xJo4uq}cX>Oq@iT8Ozvm17EyORnmV890V9DDzxZuqYz z%KvhEm+W+r)*y;=gn@x83>uITe8$;3>>x_)@j4*XqwUG_qwxL1vo+AvfJNduSiL$M zds)~!VZpys8RZJ+{gu~F1z4qS_b%hak82>2l*MFK0Qb~;sM^|c8AD& zWWy-E7R;QRzA2$yPGCco~+HV+SFPZlVS!W68<( zl^5PM|81UsnG*J3c9FPeXc}K=xO|YTblQ8sD<>d2am?X}wEI%k&m!ee(0$j1YUVBP zbDMbGr8|795n0L!C*QRhD<)+;$i+_f~-cO z-1ET8rbi{gU=@q1gH~bPMj7>tOY=L3@hHeNw)xH8r1t;v@M6f$ftEiRkA-=&qQ8`YJLAYK`qK z#OK}^!3`DB`oH-u^$=)js)toPPF zgQzcYI=AqPJa?|I{e%uC!sbWK{zystynKjjdIoF!?L+R})zp1;(^R*U+*uGsH_C(! z7C;-^cM&9&(7xys)`}8rjkd_k%Z@0n>2BC!@%69X{^7x$@(461KXjvj92#Nx`U@nt!L9^v*aziKV-1@1~?SfCH!S75J;if38@Kp`^)^) z)?C1%_ckEE7|j4Su;S5?pbZo(=7tMxrywVOxO)Xe5PJie&(Q{}6Ng;O$pZD`_6DiV z^`4~Do%i-Su0z4FRstgiiM%ymkSqCnETkow(2rRZDDkuv1i=)FH|dL4T%Wh;)xW;k ztsA6nPQDiJD2L)v_RM4c>DR#Z%NZ_nE}JYlFPYklDHW^UpwgbIcK+`erIB$ z3-<1pI`QPfv)zLqTQdV1E18U$r+x^L$RoEB^Xm=VdRz;_|8z*yFk)NNJ_+@ZXqEuf z%xZiR0wR{LGVfCDhPXABgo{_}52c+bcbBpRXl+%;aqVm5`tperHHYp@zF=Di@mMbn zq2+s)=ibPmTgv~ZlULPE$a{XeQbSon0tuI?lV4t6D(}1W{*-M#FS0IT(6Wj3yTCF! zAR!!N2{?u+vTpgHqDorfO|8pnGq^h&xV*VAP%IC+yjmW9_(VDd6u6vjNSLq?XUkj* zvXK_USE4>fVS|=X#7-CWe%JOy_MKXL&J*W`zHD;Uj#=HflG(4>-sk`+9p zj^+St`Re(JgquOlKJUb5cY4%;*RzW|>AMtPX3O=yQL4xc+C#n4_JIDJ!PoqP2T-%< ztQ#L~{69hZpVsnknI{K;!Sdl+wA;$CLdN1!n+F(~=z*Z2o7hXWbQjHqdmg!njlT6K zgt$JEYxjeAK8|Fp0AqL}!fx&o;-K2dEGezl&hW6WY1mEh>w=u)_P10TSW2sh%UG*aU5yS0#~=0T zb^Z5jS5GKfx)|w^EaR3z%h3&Q=PZ2(Wee#9t0;o3V%m|ok7%hU34x>syY%Kw=F6t? zq1*CudzLC<@&@UNxO5v;V**q%mdEG1ziqT3-V$;kCSnsGTy9K^z!+7mE~##@Q$N&J zhkF-Otf=nzAzj+4ya>o!3zruX5>v)xY&qB6#Uq4VGF2fqAGfKlQ|WC}u{HtLqJ<|< zRxoM?3$GP)Sho&3P=!UA=aA4qCT>DsYFw`&Y!u~}*#MZY%^}HnOizx9q_K^?{~@K* z;5Ebee@jJfo1CcOdWZsV9Kn?KBWF}`UbJApi`%}dqSuvtS{+*`m6(WK+^5RuuI&7wpx{#s1laU4ji zkAWP<7>Wm`XPN6{1yU@qFc_g+w$LhTE<(g4I@B;FH+l}rWEYVwRUmu^%05hUUK%nI zLJ`8F3?bX@&Yc^J(r=8|*68X#<+K0PIsa9u{Qp~>vs365YVq%ZV%d*d>J~S%o6b=z zUdiPoBXOgE>tFB=0dca!5UQThuRK6Y2KG#baWzsapwYE3eD?}?@)6{1vP*Yy=Rkog zEyLdr1lmX;;-WcseT15_)wTbMm=LD&V|Vsgmjp=0PoW_8wz|bDLahXuHH2DU_n3ti zp*(sG>rY6b72~{vj&q3d-Y#cRIh57YpiI1rLwd7SlKTi7|2}{!l*~ze2odiU z*~t3}Z*C>#Hro9Vnxb)BaiKq6ri6!5QR!qxqa>7svdkb{QX4OQL_$mss}s~6{DN05H1@&L^j>H?0UYLHP7aG++g? zu{OgO?LqD=kE??g0Vi8zQ#EQ@M~Iu^o;< z(*f`hGc62!xau(HFS+hsDXMjS#Sc}lhu_h%fdk0vz8~X#&d;q8J}e{BUJQyiQhGNW z7!$c7;ni#FA+4dBgX&&!4xRqGzAbHZhcIVp#FNVI83W{zEfR#1Z%!^$SbeM+cor0k_Nz*JKy4MX}5pS_LV_wS@ z*W$~C=~fh*9Q6La7~bjrTjsaXC(zTVPiBrV0S;1hz0Sb(Iu&JhKa9xP2^Q64^iDb=`<9;TTn@J>O&bd z1r4qd<)#IFDw-!_*Ls(Wh7~1O?@8%pm<0ON1lvJ4C@eBnu+(KZ_1%TWQVQ4Ec;xz3 zQ{tC8eqLDAbXeHjL#+YhAf}kUt+<7Xn#E38%z zyB=NPTr_(~x(>gviOo9dPD^rz?L~!Gimpa==EOt0?l@iJ&v?W4e9Bk<{M75YrbVF`E1vn!DZ1R+?ddy@ z7aKz*I$4*cZP9epOz-Yel@I8ws^|@-;GwlHS(irBA#$#U^1H?AXNot05`(RQcl1J3 z^a7S_K#DRs&{b@94uU#+^*|chkwA2IsnOXrhYOu(rUy&^0{-r;-{yhyt7;u$C!LJ~ z<5&%uA?4KIH|wAMcJ<8Y+o-E{xcEim3(O|6JoC~ix^ClLn;X-cYvR3|8_Vr5Md^7L zWG?+xH$#LozeU;@;->pLxP5 zxB@b(_Vh>0nOE({Ze*s~Pa0C;7Aw4Z1)r)4ZZ}(U>$Pmn?mXwc!59^_1}XRw4pQsJ z)}e_qs8sHDobVBw`k-#a*0dii12Yq{3?pF#%zU_*5zONjSCvd&tyBO|TkGkLyW5Q! z0`GCS8Hg@a*PL$<4WTzHbG3vJU#hKEFR5)=zwhjN@OhuJ^_SdCy)*BLQ~OumW~_Tq zpgZbHuQAiE28Z4-Bs*bpN%cTd2CeT#Sd$VS$SY0LD9mOWtXE1kR2?&4zdxMjD$N29 z#`3Cv33}{(bmv4(gpPeHH+%a8OS#~<5W~9azNgUJ5i0u8Lof}1WvtxjaaggNSAAgm zgI#%u-fmq_@r$o4o9@G2Lwmh0EOdb zM~jWbKCi8(sQYyZOyrraVwgLW)$%+_W%ypyX*(9dmC7$lZezEGYZ-W5_2Ely|U*p2C3M6=XM19%0 zO6wj+!)QaToEFLu*;(e^4fUz8emU;I1?X%W|5)utM|lVG!kw*$DR%Vp=8YTT_mUD5Iy zGwKRyGy~$pQref&y^sLeu+jrgU_rHqBo77s{)Dwr^skl8yV;CGsjlYW|qXXt=4?Lqln97z^hwfhlY6bCj?my&xAf7 z6Xb|FyLzAP0?)*Kfdt^m7z7;{!sdU&V}wom%g@dsL`ID+qI@`Y-299Ka{!yzAzIkT zo&54Ez;h>wf3H_lA8b(9(D~RxsOk3e%xBSb@+~d_w_qlG09`|qOD95H&ik^cr*^<5 zo|tgYhii&mnH|87eT6}EPo-{q`i?jsKdG!Qkci*zdn=bk)byT&qdl}=7B9A`F1E&I zu9+tWxNb868isXZ*@U|^p;%vC%wNk?Ufw86Oq@(OO_>`OkKUr1jfwoPRWK*7bkik{>%;#`yI{x{)A%9By5hHs&zQRHbo2w{OeDavqlQlWxDlb z7Ey1(Lfm8C4<*D=_sfP6FQ~}co@1cO?aB(V^D(gf#}3Ik@-&EshF#|G&(0o?X?~ zXs1eFdr7~h0|l;Sf!>pNFB0c;gxB69U?qPY1iO%TbcyV8-;KLr{X+3&>BBGpk{$X= z0gh})o!1$r*}qoSw%%O&OzTUBf-a?u<2_R2;%T6YP|1x)hQg1PR(z?Cys>XPR?;%dZQH1l+PL<$%%5%(sUQ{lKwC!Ew*DRT zYDz(wG$9)MY%;3YB+{$@VK+bAf z%_hm}T><1V)xIq{&4)DF9ov@*)$!s)-5X!m(@w;y;(wuAuAlA( zVvC_lNY@ug1=Q+dV^($Gp&%4pz@^WyWWs@K0LodSrs0rLvbMsL5)f4st{VMs?dbt$ zMYY8m`l zA#}a)?+QBA&wR1L64trC4hu27^Cgy`*5_JBSuYv!ajwP-%WM06ZZ_qWqh9&a>=#Nu zIw>GXi`gd$RXF>5(ec^b8Z79n0ln6lL zUtiA#GV}gx$zi7g&kP0JFks!7aUUO`_x3;eqnRN|R2?D|i%!Tz(j7$1M&5IUY3mg% zt)74`RY_&{f-IppwqiaZ#rB~^HZ!sWgvT$A*Jh})w9>7`tVUQvhe)CDG6J_7n>((g zLvh7MrfBWU=!%IF>4Z~BH_2;9cPFmEZ7i*Fch+%-SbG^4&$82$9Ut?yZM)WTJC#L1 zTeeQ+{;3HgHrZte``Z!ug39zUj<<0PRG0H$Eb3|II^0VIRkJ*fK`II0N#&u*Q@plh z=E%#6eRtYEr#RcYJ+Bftq^7D95xRglt>Oqnyb{f>%5~>hDXZqbF9PlRn!jDYpB{@t!jErzEE60Ja>@zFyNZYR!&=rtk*+$$<<6M_B$N!g6WZ(jb~{T zLaLl;#9mgES$3+fuL(#>Zz+8GNdic)&~2U5N(<)mw;Wh=Q6*@Z@0!r`@O#6fE=|{;2p;~83_^X4&cgW-rcXUmpl@-@fRj_jj zc!F?^rbMP&lRFbt^q-M9ixMip6nD2&fR$q}6RTV_fz;mOH7Wfuo^orLkebe~P3GG8 z3hj;np+2|(yYuc_Wm&Qt041J;4KpXZXGoNb@Y*RDjfRPD0S@*swdf#ioZRFk+L0Y# z=S)?Jw(z&!3TB*@o_)`1H*;24w{{Gdt(c~AR>m=vY8b?skNAJ z_scxcbP@OjaZW~eH+7wbFg!IB1QWkrX@k*JWe9K{y&1qrrHZ~Ql=`Rd`a_Gyzb!N2 ze=Y{CF3b)Cv0-B#`<4Ll#x_++LizAoDvJT72a-TM1oWl-CDp4JH(x;u6XE}Zu7TS& z-f(Zlvj!=lz2@Tgdf{*8e84H{yUh%GF$f0g0bGZ%9@K%GG3zU{*F>RX2|f&l%_Ex5 zKmW)GGh{G>pM=Sz?*{L&MmQ;qgwe1b)qNYGA7K-{>H}nM9vIL2Uw#1T-V!B=U~*gN z`q7fV3sX}SBUFj8!dvB}3DPb_h)DND5FXrERAas-kO-RSSjD#LKW{C1hL4xv8A9LC!m7K`I|rw&{o|lK zu>}jsc>Eaq$9_fNRitZe0ugf3yS?k>mb0S9OAFc}wfXitro3zAV5%>YLDI`4{K{61K0?48(>f;|C2`No0lb2G z@-0KoO!2q-kI8x-Ye3$#YC*2ju;MMa+VVF#6MtS?^tPOCaAtr?LIHY5WhD2)vUbp+ znPeZWqbVEbUS!d-*0V{QtC1S~mkT_!2NsxD(-dvc(-Kq)rJ3E${gUoRfS-y+8_4wj z#kM-n!G+ASoFFWH{gtDCeOb`9EbtOhwib4u4xs$2BTV?j|Dhj)XS2MuZQJ(H tOR{H`|5v`?e_ra}>_q=B+wc{~?O&DLEipR_x4}P`

    F7&fR_Te*g-*-9G>T literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png new file mode 100644 index 0000000000000000000000000000000000000000..6064df3455b5468f913b67b35933f4f0202b047f GIT binary patch literal 130427 zcmZ^~1zc3!7dDCrQi_xy(%m3PNOz8ONTY->q;!K|AR^s0AStbM4j>>U-Hn8Fci%n8 z`~Kg1@AvWh4Rhw4oolbX_OqU~PN0gC>^;m!m`F%S_vGcI)R2%++`wl6{SLT8q*dXG zgmhO7Dk-TVFDXf>;$Zg*YHfyuBo`R{98Fujoe-uM5hiAF3nk;-G)nj*F$HwCH~Z|& zR2Vo4pZ!veWeFeXV=)tc`;tffDfyK+u3(VyRuJj~GXvJqXg;RYHTZ7b3A}1=#<%-? z=~va0qv=~>8&73vJEVthwc!(fH1WS!k#A>m>Q@&-ITa&WzJ-R(XVm=p6DHcXyJfx{ zTZOl!#tpy6T3(b~)f3($7fr!J!b7ze;7CbC+Yv|Fwk(v>M?zBVNK|CQk>|`8gWSFS z&g~0n*+%RL46`lr9VQ z$Ui7~-#nFzBf%}49e9EwZvF8$rg%06=BPc+nnZ0_3dO^`Z)3DD()8{3yrv&@o}v22 z(8QcXpH_3U2bweQ8=3EnVR}C+V`O2+HO6fI^WOg*`bSMFm$sh3d+MW~**c7+?~Z+> zQTm|NWcoEz+p@gJp9b@IaG7m3+KUcYg_9afvRoP4|2Z6r8^LJ^^XR-SSAll zcHSXZQZAmlefUbrtVGUW7}bRl9MSrg}C2>-_T60{xmI=EM{l1E;J^^&Q>K9yt@jmaVW(uDmr zH^kpQ?AasRHeen5iQ!;Ui<8RT!GF20 z^Ex@GQR6ZGuiKfaq^!5|8U()KTHg|Eu>D5C?N2j{^7J128_4XV%G=MC@4MYPZ4}%l z7RSMpkhb_pZv0uB(&Z!m4>7MWFc8H8g2+%ybf zj&ei}SnhB3sf)su89(Hv>VBoN3=EMWX2icIIp2$6K`(&HCIRWCTYgk}XX-s~Dnl>U zGEXUqw;%QkB=w(8V0h#vFIZXeAdOs)IeAG~nwY;o$|tw->p8(34nBBEVNb2pv{EUo z-$>;q)ZLOjTe8z~EZ~JU{ib66`8JxH=w+}N-9wypbQlURRbZHWoB|`A23|QXHm*`* z-JqnKEG?e%M=dGWN}|ZXv1a1tmga+I;pX{f_Bon4q(G^Dnz8U&`EXk7oHrT@x{z^c zOF9GkgD??BCKh5AekN3=Oh%08PtnZLF3e==)%p7xD^GNit;G*%Wh2}dRu(cB3Kj@= zr5)~{Jdu8DEWMM}tXj;p$7scP5Tzbf6UEoj(UH|b-qF{Ayjay?6?Md%qUn=*t}3nG zqV+yAB9~fKC#~3Ev9g0LnoomyC}LGs@e83!tcq0*L(WkSg$6ViJJ(thwn{q0JalKM zGnZqi&63daSy`Z^v}Lp<+8}%GK~Cfl+mOyG>A~2l=4#g}-yj;d#Um4LYMxzgLaX(` zr`bEtA~T>l#9!M7So+sNh}f`f5ERG;eKpH$8SWTn9vLqf${8vu z$k|9sP8C+HQ8@eRQh}XilG8NYKBSXdVzgAw(Qd`a$-pU9V{744;}+#)#ZzTXFcA>@=^~J?neM zcV9iU=3IXK@G&>nU0wp>LLMRLsCkCTf_aopY5#JZY7uAGXsh;%+WyL+V}66g?u@1W zk=}`w3;H|sRP=a}UG&NHDT)Qp<`sh!kK(4Eq{ngo;`*hPP!@lXNRk-(Bn4_{?la+U z-T1AoSlh<_=UiQ8v{~9Ba;HqExmLB7bslw|KwizThLzD8(=gBQbDNL6cN0ky4|rp( zRrQ`vuud>f1i5@oPE2mFtuq*5{J|Ki`a_i`-F7r@^vMR+#@af?)8(h{1PKJW1!_rc zLx&fe>qRCx|x6e6W7bxVxs%A)}`2aPC%32z&T3*jlaj<*1>1dcipe4YIHTiRl;#dO5XQ(wLCnyqd)6VDQl7k3uFXe?}e z;vX7-Bk@BbCSdXfNpaP>u8Z$i&*l8y!^hFzM!z|~yx(Ng)bg8czSsWCyw>L$l9#wt zpDjM4gs-)oTS_LugM`ORj%R><50 z-XNYDz9s%N!yQ^JS{H_3rG#iFDjiXsu*oP11$nu#7_P@6ug`OLgmVgWGIK^4v=!rc ze@{wX1XkgPDybzl$2(C5hsDH|gVkV&P;VlZGtYjWedMS!IW$QmBs#5N$T5>>)aaY= zJ6Sb+hjl>EQTml#jGdO5R>~VXZ3ee!vWPm)GjAo+?!%vF6cLjH-dzXab%aCv0 z#eECV$Qmh^G&R*Df*z*4to@*-=84_#bBq&3RNZY^xTG=LiS%x2Z}zv`?WVzZIcQnX zXZT8Makhfqef!zc;?nh94d$=SOS`z^`R)F^)%m59XiifyU|!M1nW)Z{ZEm|gNw&wh z=kVvka9vwp_1wj0?mPysDWZxl)Cc1-vrwZxe^&!dnyvW>FM zvNFMYF6;%8YqZXFHEON~<7D4za;D)vd*sQ&i=K^!?bAzV{f7oK6y^0BNb_PB;z4hR z0xSYL9)*2axpxt)6FjBZu4vHprOSrRkSzdPkaXIoc|mSSbt4@Tsu~haN@C4sN%oXo zVA_>>uQ|P3-o{mlKDlYfh zd4+>rUA2MR*DlTOS@s^?3E$cImEEkYEUJ7Vaa#N*uWKa6?&ZuT_Mdy$ zpPm|rtUpJ3`=F~0Y4*Sc>1#Vu@9fEEE<>No$OlqBucE28Q@)*lW@NEN8f>~cS{6BW zb-ewiPVe^))@oiRGLAGVW%upW?(Qb%<$QX<=L%T3u-m-SUWz>sw5zPPo+Q!c3rDPs zT=vbMMAkLqXfPYe_`N*sPp&xcEkrHH1G=#eB4rz|0#{)mU{c&>jm%(NnBl0ULJg_ zn>d)6**aR-Ir-e3ivl+=Ud!n?A|VmdAwIX{)o8at`=d||Z6|HTX96a6Hta7=?TpRX z-E3YX+CdU>69AVsW==0D-E6FF9R=Kksjo{2fNR8U4rBeU3Nb|Rmf7+2Ub2M>)zIKAz*-|3feQ9jx>?BN0jp*p#zrW)&bA$e` zCtJsxX@LoHAf9k=vOnhdw{1{W2ys_H1?pyIts@1s0Wt%9h;VZ8a0y*k`2RinUzh)> zs{Nm;oZQ_1t@@uw|KF;bj%E&$b~d0>Cz1ax*iGgCKD?fvMOL9Nug588sHnK8R8Gw0sLe9`wgyd-D0ee)iy;!5<`-g64!9MwVsSt zNg!SQ!50TtHSJTi+~c*}xRQ^B&@Mv**CO`^w~f2`mqQ@l*Ze`H`T4 zD=vNu_0Qq+nxP7?r}>{aTK;+Fw&spKrEaiO0D~2V)?Zr22KwVtBHcn4Lqf)RgM{** zOUoEBK@?a5ncySr|I=gATPWnVD1YmTg`@afo|q>-ekoyDh)pJ-y)|8@UWAMBpL%`{ zxMH(RL^>Vs5uNx|)4gd>uaJba{@bBg7oON`&e)gq-`C^l)k48>8~-G9Ao0I}El|H{ zhzz>B^LL&bD8+d9LI^}jLu?O>q>rQkLo#fHT)=>R3L9q)&Co|tO&9B=~ zp+`m+GDC)Z|IgIv@WD8V$%wD(q^e?2Zga1;4cPrFAeB1MZLX`e1@X18G?;Foz?jJ% zCZPU1*++3m$jZ0xix>o4kD?!pDe7mi{(t&dqyYmHDUZ5)UB_7g$uC<8Eyh;-pQ-vi zq6Y<<-pUUA9fb{ITK8NnuVweQpEgCx%7(rAtFwB08`o*~Y%?z6321h*zJ86%#(Ie0 zrd)8MuxI&3?Or##!P())Mv?DDURMH7r^}cb5fNW2spIHI#fQFm~I`s-^NoJ2*d&632z-Mp-Om zq{`8}W+f|eJv+sxxbZDoO{M#v_*UE6ou7l1IqKQt)1F%!{au1vwJuHf_(r3c)U8vz z5AE#@s-3>C71dAsobSj^xXySR=#LZ{Y*eomLW0kJ2T)koUmQBFR6DP|8Zq=NP=pDM zeYi_JH&^YtWeWzWXgPT_<7+1wfbG2EakvhBZr~i9*`k>Av=95~EK;m3{Nzu`%cc)w zpj~q`a<|l=MBmGkjSiNgXTdtj(Uh@lnd1@uN-z#Y_DS^di8f)kQwVXWOncw)v4fZn`fz*>drRul{=w2mC3|Ciwr2CRC_DQ_nP9di@6$M`>wgkLcB8h`=gi zY@u?Bhzsar)I7!el`MsKY0%n4MNu*sY>xfaJ<9)u<>M|F|P+dtEqy3{>Xz$lx<$clfUL}ze(^%V_%$VWiX50v4-PHf*agtBlVYN*)WgXAiY=dW|Qu^lV@A?Tm3A*i#yrZI^4JF zj`R&aoo(3pu8nVR)@+RvnKCQA`lN}N;tSxXt1ivB zvf8?npyu4BRi>UvFwf-0pD>Fn0%=bQ#71wWpB{DFwQr+iu;PDqendd8Vg3`4N|9a!+xEcl5>Wqe;)hG4m-$ z6?+of$}gTGSuMhQzn*MXIr?9j;`jxbzsc*p&9kYyzz7dzQH3sw95VyOnb4 zax~?d=)=G}W{Q8b?z`L0Fy&~p;x&n{EcDxlf}jHI%j%in(#lo}-^q(<#;I3{Hh+8q zE_XtGD`)6hE`W-;)<5^XI-8l=ik;(~@p3$>0hS=NYKFhX=6eg_<`D>!yxJ4JGVwhfNLjx;+j32MP;qfMF+p*4 zEL*b$KU;@+ZmLEh_E`CRh~RkL(e&CS!BK$ddE-T93scH23uDTz6{%*Zf5&rS%YgCp zRF|*9)o6pZs8hFBA-YM^zRlv|Theo>u4FrEy@Fv6vszXzWX~V$8`55ST=hL=TCS6u z&p4t8&i0=U;)Z#Q^i{GflgS-PAgVf1ARZrugRA=Mt3+^R#EiI?IqmFp#f&KFgebZ1 ztA}%|q`4(SRC@9z}gMVbi_@PB1S9g`Z4y z)7bmx%^rFuz~~=po1rQCprFqG80l(`3v0h{b4f+t*gXE z{NMaxdkMF}xcDUZ%uz3FEiUtC2)pVY>?fwX=9WIaV24f@rp~Z*O&pwMp49pKjBwYF z*rhljphv2W$=qoH1GD){zpeRIM@y%2$9_u5tN-$lI$f%Np+ z)s^C)*4>uFk`L+}R@VQZb*@h6*IK__o@QWrgGe;kZn3)Mx>#5|{1-uf6}ujlp!l$P-ZdQpgJc2$yW3~{s-z<2qHJKxs`T5 zTkAx>Ddd4bc11Lh`v1YlMT~d-s{U9)U!DaqhekcRfqd@~sPi*^&cEf5P;}{W6lBF` zq47LSH+K~Qyy9+bMEe)xA^B1`8vD+c9rTs`KMbR1n!eT3wiZFX)x1* zv0<=pzta`edjAWG_qUA(Fp1wX|3T;I(v(pl{tft+QjBQVf_?M>2)5}i58403X2t*n z+rwV7ZVXOM&#_x7w{GT9fB=&Z+*kis^gtFghd1e;eW@})bA#yB!7?biZ)1Sw#WFE! zkET6&q5T=PLjW@*mH&KXwVKG{DU{cKPAbtH=AJJbN)DB0CXX#rtkN#E8tF#hg9&g{CR3osM&++d7`mr+mzAfJ?&$EG)OH(pI7i))tNeTe_616RI7w#xT zGANe=@L9c;4EGHtkXT`CM1`qXwLXGAUSYWw+#n$U`s0r{bg^!s4^rIXu(SpTQ#m;B zvt3lKR%&-USh5W}iB14Mn4pW*H-3BPUh&1zjHprnCxgzBENj9N^PcjZ`PK;l+Dwb; zFRPt5#)p)I4>ZoY?0nR}9}YW8pZrW4_31QTT8LCGUm3`r;GM8GK|r07>eKa-o^=3= z%Q)c2KX}J2XeI#s`@~kQxe_Y0EBE{;`ne%=uSe83uY;L>PmyoRR9R@}-P!cY*@zq! zLQw|fT<>{bueTp1khK4c3>?HV)8TKY{Zy4DGs`QiwrpR_FuTj|r_53xRq18Z(K-h} z3<47CEOjSulnyB6Et}Rw`3y_Iyi8P|on=3ZHwF-))*GM+1fU#ybL)13=@q~o>mO-C z9_y6Jp_~FxH?iQsFAChBNL!CFO_uF-fYx2WGG$gIXy51=84lPlZ^bjPfSxJhw=r0n zx~{*yFY;JNinVgqzl*-F*nybOHVI#ZQ2WI#&#Q~0sU_exj4#|an{DtLE-_bTpa!mw zCpbo}5srrQuudy?-}B2MExm&Fg)@MyOO-`026g&=@y&P@c3)ncne(w+o&(%BF=1D4 zW8{4hy;qmEtC8)``gFHU9;$1+GyC@bIGO9@?^8-~R>5DuWjdHIsF>zD+0t?9lQu|P zY+_{l`1UlL7xHd-AUm|6Y*<$@23IleapSbto}xKV>7+v^50gyi%(UD5Jw1yr?~__S zv7bI=qti1EqpRD zb8*Ulr`_j-bI1XXfRn*>9ao1Fb_Nq8dUnDLAvx<3KP6V0_W;axI7&KS`I^F91Bqp? zCJhLuksTj;ZnQDZLB+t<_DR-P&!#LI3QNBGK4b!{*t&LM<=m(41PI(ml24$Iw`;6cz(BC>nH+_2GlN%#7JjrStQIE_Onv$W%EZ| zC3H>i>?DUzHs2cWMCXnN+o6-uRiY}lpsV2C7@A;&p}}>_MJV335i(ogBYBmCr{s>m zII}V#Ly=tPyG_`FzfMH>cbf61P@Z&|htf9`r-p^zRV=P?Ui(AelJJ@N9DvVK z;H^$CBrc9y6*)(Gwm_W1-lWb03~2)lL`m9wD#oe%QnERoR>T+Jf0hGmT}@|<7X?=I zMAw+g-lQ#z>adfezVcB1*S7fFFmN({zk5WHo!%TKiOrueR?LpUY5c{6OO6yDB1-Ig zF!Y?v3d0nQ^tBjQ&R%i`FN8ncqdA{I~} z7<`yt^(Vo8zl(3iI#l$sagWb0vJM=42tEs;Xyc)X674&SUFLe8 z5hLHal4=0+g`X+CGV}i3f)ve`_aT^XdQ^vae{*UAg9_mm3tye@vCouG*se*TL}z;V z0zX~w_d9&!pa;?RLz>UyS?Ov0*8w(HLU99IbsgcAShWI+VH?C+EH}-YlO&;S!4%ev zeKMy5T*as^TA3uLetD(Ilf9Iy^=7mae|O-mC~Aook5&S&D>*Zuf2~4U>)tL2&3A=i zw&Gd9BABlud=D*^r4(Oyi5rvVtg3Kv+n7TqTk=<6po`-2_f*be+%!_rdV999 zyO?g~;>#}WyEcDFcR1gale+m3x8)!s18KYQAK)_c>NqoftYd9puicgy;ULm!5N!!b zBy!UzMak;oo%j`8fYJfHl?fgB_V$u-I@uoMOiyJ(GrT+k4pkMAf{qrgqi`pbGEIpf zxNEUTf81KpB$eXijfzRD|H-Qii0|S*|$gSD(#9mul!Fx zIl{O)%g%yhquXK{_XM+D3T~EgvEsocH7po!A+fP5l?I{56XfbM6~WhuHKpF6jOWH@ zE3#jqzLh2K#eK_hr>{8rnW^)@f-WtAsfM}fMtGl{6xSr+gV<9-M5H?3=RQo9+0F^7UAa_Ge|KTx#I_c{|XSl#P*m9-`ORD40lYZ{=B z@5XUQYM69PXV)jhtD&-*+A8kp6gaci=lbbG`gr?@FE}rZ8!GdcxaT^kDHIc--Sx6~0 zT|c$+xE{CtyT1T^I%;7?6ekA#YWeFfGIQ~IK8vviW!zZ~T&!38y4_2X z@bA4m8@gHcBjx)n1*H*sgmkifVnx}DX@XeGlJ-oDeBdTb^M7J3pv1745W2Un zbNT$WF)b%yRzw93B?Sw-D^)RO(h?y}VH=^c;niB~t(^$(0Xeju>FEv0y&4Uas4j%{ z!XF96^1T>UnvAXyZLJ%Xm~oI)?0T3L@=U}-N?(ET2l8+&+NW*yMdb{!9Qca#k;tK5 zNfs6K{5mm`^Dz=YU5uz$2cHMt##30RmvPR!mzQ7siOhSyFK1}0{_4_}fi93_bpZGV zbEo(^IvH8YGUd??_&Q?E-T8Gii>5W_RX-VY6&{?;w$qJ2VA=la6A@k^6Q zl-GebdGfo0NPKBxGzI=lxmT-ktKysnqhB_jH4EML0={eQ8+Ew^FO#8HBNB19=sJ7H zCI9iEU>M(|@?f&Da6omJiBTpH)MU~@J6}7LiQ-vs?eeE(astCxqc{9uV|0+!y^!wd zNz9&wjwbCJQ)-sjUx1t)$@pcVXE`#wfX7yE%4`#;9r^3(A}mj_dJ-=PL6RQ01FkqQ z)+8rb7Y{Gc06J9VG)KVC5mH3qHtWZ(Oa=LLUYu*}y`90z9xEMw6` zq^&4ooSa|kP?bOQB^fvf!=)8dCn0=d# za%}2{*d2a90nI-!IQhJeE5e&D;?jLn1N}F5e8Qnz1TuR!xH}O1y31c zvC?~w7C4B@Mt z*weTUrQXh7OyjEU2lG>|Gr{xcfpdeeIJ8$vPLuvfji1pfeBy`0O_MlkwEbH5!Xg)9 zB;cHy)5a(;3t3iR^)5QG@hmj7N(FVi+GmKyl#=FA`T|`aZ=lA331>8@+m=V;xTTqe zcb#u$1Pc0)_DjLqI$DjeT?Q|3sVTSLrOFXsd&a#g!0~uwS-HZBLaFi|%uajTO7`X;z+SB@(@S!2Tw@h&yA7E;=Y=0osE2q;eYJLGER?_lT^pNB`9 z@9cjWwP%r@-Kj}ifh&D*C^`6Hu7(a7rkK;8dsgbjeA&scd3;(dO&+YboO(tng}(V# z){5A!YB7da&q1c;hD30{Z$MM7!cZ4MD9JfW?F=!0|HfE}d3W=!XBv)8xNV=3GJQQ4 z@G%|iDSXer2W<@(BuaPsKJ(EgNDC}?gp;!EaPsRKfB=GZNf&SDS`i(d+%pl)pTQ*N zQ#G!x*nTvAEolW625>pKZ|{i=N2yhthQ9ccd#-aXHGuQHuuu{3$)VDaM18?U2H8a> zC93C%PHc(qS0hTy`5$SCoFs-1O!EJUlWd<)b(%}#?Znv5%`3F*+htig*MXY=k5G2v zWdPaSzChKiKNf58XM3Nw!ypP1^zLp@`DM9WhC_aYQr@Bwqb}?fBCc~Vtk;tn%<8GH zTgI}Y!jJbvUo^rVK4+ZF%927#7W#;%gS#BWe&8gLa{1rnQePiIOm$2V9`7`#CO=ff zZ8un&b|LQkevb7CU&I#KNtf!ClaZ8vqF}By>d9Il3Im_k#o3XohzB>jr8({D)7D=B z6h4(TDVLE=`Cz?T(3DMu|NPmvjNHUJ3GF`;`;uf`7s~p*zmg3Q{DnRw)4!{H{j zsQmm#BCfj3HG^Jk(Q8Ek*NhX~kMxFZIzTWwhO#j}2=gY<5cG(x?75%_bSd8g%Z|?- z_*;>Qrr}Mq3@Kjy7&zgf@yEIgL@m@13g6;gFN^l}V)}(~14!Bjf0PFs=d|Ip zx*z6UoYXucBp{~33sNw<`itSk@P}za3G40O?YT0_dE7T5JlD$tgb3%j5K#3{R>2C|PQV3*)Ov1zb07ms49g ziwL)zv*!|_5>;azITcA7K|wBu{hMm@+s9f>J2CwZ(MMM z0UmV3Tiot?XsO6xftZovng3xs--x|MaHr-)b%(`?w$(Y}AN*{uMdg3*HvOHUF5%mK+vAs6v%*%OYu2v2QD7@@jqxO2B4)W zY&Nq0ajg#+{>Pu%r)m5ZGY%sHoBO58-AoJ7VlOq2A11PL&{uhcfPn}s6ev>~(xWa; z53CaWl>Z$fLO9~7mZ&fgsGa~mOkDby)Xj2PrhsNQLaBri4gPIqNaBV`)kW&VzG1kk z01Xlq#Jc$JvfM(CQUJmGCuSW4YyFv!oDHgfT=G?fOMZdVC-%20XjbLPwS=y@NXu8W z7L3;b=HA>Zcrd!lalV8F)xJ0~1DJh0{{6|?fZ5srwatI*HS zX0E!O1KqAWvXDeZ2(4*6=buarKQ+WwM#SmY#>=e!&aameg?8I(6na%#C%l+j8#^DA zeJ@yf4@RI1!^ALnr5BwICm7!6xqMy7 z-p4S!8JnN$>|n1~34zzHpTqTiS<8h|dME}uI7(;UPtPc^?mk%Lr{79cC|5r6%^~%6 z_42&<-n)Bzd9ifzwqbsYG~};H{W|{cAPw7JUjex;hw~a)*|af|fcqS!p!aHJWPy*Z zz9?&SH;Mzw_IuAfHoND2`S*ejmdxOjQ$4K3oD0Lxs-AnO!4SxWP3P|5VegS^S!}jQ z#n2Jm{*jJ1Q~9r$%~plzfQIxw8iZPdb!3Jh^cOI-BZW1a)gAq*ykO`3K|gQxm&H`1 z?JM0=Pjm@yk*7~^s|b3RxcsoiJZ%k{qAV!ZuN1|>a^)-HN%}rE`&3D^A@=C%WS~QI zF*jynq_p;G%gE(yzv_5*w?bJvNm`cN2et)0<6BG_76}d|iS_Ty^E#tF;1A*7B$_dr zQtnP`;^_-@Jv}BcH1xP!KP&0ob*c}5*C)ordU_pSW{TP+_@p@Q?aidH@2>WDwQ4#Jq9D=WR~gB_p9Ej@0Z1#q>M!|xC0hjNkNDDL|$F+mx<@f-daZE ztJ4_5?C)d1kHP>?TtC@#xf`7xr1t|}S9$QpaDkppf9eo0Yc1%eXS4EqC*i|!#$FK_ zi^@SPgR-(;I(^4;h3u*#hg;Vxdr2p9YujFD&zkAj@UXw1=6jT0KG$}{aYnLxm4a1g zx4L*1(6D!9((^YjNGw(DKU#_!;epUns<8b=_2sL5tKGc!IwHsKImc44*pyEqhViO8 z)T<>rt@CyjpeFFkVF`Cj-!F54KQl8g%gx;9$WAW{4|fNb%1`0%E=BSTj_+hwNfF!m*h&||sa*G}p<6#oQ zqJ70f7D~)BrL?&7w=Cd3fT?beWY9a>oPtVV=h)D5(0;WCMEP@zl$G+pqSVB?w} z@bjSlpTSmsD{TZy{{aKxHkY`I=%}2&p`@xn_RZ-X%#*?ytZ0bewz`SsBTaPZ}uXXdo8 zoRFzfx>k5_KuGC&uZr^Yk*vS;v^icfC0Ig?^bt(VDy8e* zeH^5lapR$D6f`_>T2`r9&COF-#{0Woy$SNojeHa_)ufQLV1KWzw)@JEVd2;iWYJwmCJ|?ppYcC|-{Fu4Snu}1$ZT*K&G0z4qwXa-am7}qOI4x3xb0=3 zJFNpKY!slz9Xgeck}7f4KARh{KL&AJ0uqkoBXL2ykm;`_rP^$RMCqeF1W?hSO#5BN+1zZ~>BL zWIjLq-61;r=C(CRNn(Q-#2UE7dtaU`4a2=X1p(LIsV1vOsJmWL#WtZZb+eovh;VwS zVeW@}Bw8FvD`)^g&?C1fQ5Sn}Wd!oHXT8D_g04hDJE*q35Rto!gOLf~Mr}!bh6#I) z_d_^r5Rgs2p&i@zq|?YcRQPh95rn0R%Bdtjm@+^kd(bGnz8ToJJSg0JZsZMuY&KU}P(CoAnU2~<^J>7gRy?3J^~v0DBMAk?U*0yoAl`2O=z)|TI9 ztC89km^_hJD`D? z?S#QH$%pxAZ&7sS?ve|QW+^7d=uS8I-B$Ehy1KlWUpt`P-d`DLg&)shF^tMp%6|Y! zWBu3y%alYzAk;MhVxEfB4In~A#NfGBP+<)|E+J5M=7b>en_W1NwJjF_x!Go*`MoK_}BxYL@A} z?!+R~;n<`LTjs03I*o*qfZK+GjZgm!U7xxI_@f7Ml&2}@{xt_bNBiOrl%^>!1(&lV z^?rHIKjEv{??_}h=7z6(K^>#R34K9`)+{Zc;4Fw$n=2eWJTRY=E77K0hhEp8z$l3I zC0BT~g1(gL3Lg03Hg{Yu64)G9X%N0BA>I`jV_zi{F0E9u4|2K`^{3U9g@@Kp>n{(A zp3$B+LqUFC%d`FZNAV!(lxLtoui~lU5`gv4lJn?~y%%e(ZXn!(X}AdDl%%l~7(xZi zqIf^%imCmwHO-Cb99-q=DTH_hzEnpxk{{{h2Omoe9OhN9v&m}#fw{K2Ka&$zt3P&I2 zBI(!o99&$Wcm}wPuZYFU-!7_LIuv(yl||(}+$J(~`yw5K!X9+q3#e^J*~I~!vNW}Bi7)+bAX#uv!o_<^&Nf!c^gI2!xZJfl^q2k zdh56Mc-vO;WJGLbf*+Zg{QS&WCVy$eJNa5t&ugb8aWK3G4X~&uJ#NI&UirtF3Q$1k zTalfCS&~tZff$QnWDynW$pIohr*E`qBKtBkTB{f&d(`Z$`%;IuffNPdB{XTcPw5~F z@vH76Xi@F%Cq!yu_x)Z>mi$?5r;Tx*_+QDLf%}6w>IKh>47)|uOvO!CM?hA@0_lMa zq3m}PC_k3Z@|OVCQzJsb^X(>sOAul2EVUIVW_0N6NYGS~kfMT=I?dFA)Qj%lMM)ZY zq1B=bM~uV{Y<-{dQ`UibeJ#I;FTXLO_IS#U*#tx$%L>ZJnRPTG9m`a0+&~n=U>fA? zPJ@s;R-#D=e?kUCFO3H{V~FfS0|7Ss&cj%mdT|HSd`e%D?nVT$@GSsQ#2_f2PK@TP0R=k!4qbP3G1h!#G@?bB%DJ$lTj0C1JT1<`FY3-n zIk|QzrPwE;SLY>=*wWd+=C_|?=%q#R6faFLB1ra6#o3H6+xHvIR4Ayd9bWjfy2NyCIr&&6@Hde>7YGJESG{nhKeVaL}ex7tP)ob z?Ze-$R@RL%Fu>f><{3WpN!7P6BF=(#JPsp6kk0=k6yy;{mY(8Z+%4>u>uqZjXgg|O z8c_R&AhQ+`i5c^UCu^4v`6X?&Tyk(vVk|%kHh&4ougNZ#Jo*L9+C;{7KxdMBVN7yANRuNS$0io^dlq%(w+ zyLH5Q3z)?A9Uh}C;~=&_F=D4ZV?aSu> zK$dm2TZ~0Z6QGxFob-b0BrCX7Ui1aEAV3t9D4%fM-qA8O6U@EM#`!#ktW4#utAMZb zv9jIB)P6vU7Rr=T$iOg}@WyhNcw;m&d<##MYKYTeb~2-~ZJa9o)GMLw2jx7&&?mju zji^t;-zy|B6nnHu4L`B>I3eg;cz)^cMQr$5HH+Jip*W=d*OMM0_PUVMq`OdL<%LN; znkzw(lylsbl1pkU>{x>(_`SYOlXhw>fzW&_V?P=1&n~Q;a!!y>xWMX07S}e+uSIK@ zO?#~I?8Iis*TqCiYta?fB`lxqmuV5g(r<7l8!&mrx%Pq5%$=BePBs_@sq@^(h85JTjXrLQn+t7`e`Wr zAUwIR_}f#%NZUY?YI>_wBcGENgHA%XsBYzDp7t0zWfwVoH?IT>`OkSK0AKE@?x^~| zw?pJl5w;RJr0q|CVj4Hg!>G5&(UKAI)hI{7vE#{fh(zwel_kZ+a_o!Xiyf|z)lwE& zzC758b}DA2P=*@@Vz-CwZ!DlbG2oDz4M&eFZHSN#En2$odn~CRO!~s}`!l5NYR868awd`ZL zoZD?h!4AgMS+M95u6 zUveg9G?aksjT$$d5IVNRD9)-$$gmM+yMt~0g$q&ad24JyLuAGe{(rYO)!+QiF2*n- zWE{qLvG2&C%nIxS{q8x`%H`S_#S$nRVT-8m%wi47m#HI>mB)Ekbg|QS>AoF@?dznQ z(eJiIp6~g1nJ%W|@Mi~4Fj_^gQ$p@4ZXU$GYMgJuDU%e`U&PGB6oS_=*NaR$x=w?$6)g#c^`G2A-5{Tfi%x%;^H8Py_)g4 zdO|Y;o12#**&?IB#9WK{KB=j56xl9Y-#AEWnm?5P%uIhA=*oU2t82J0QGd2s(<(iZ z(69$iSAlv7IL9R;goHn^d)h-Jqk*QAdx&U5mvGN&g*K`ovNS)?<%gw~O+o{Wc_&O6 zBFi?TwoNEd9op|---F}Uzl?+Sk59~W!x1a2+;s=e+fN#mPYpB*P~_kky$96$3mqg8 zKBbwCLSOdbBK;lpAoYu@7`tz#6gr?Ep@u@vKYM^bG6P`40tMjE-i>K&;4-E&8x0~)8?1Q(!NLsy5SF%N*UIY{= z3@ggngs8Ae^(*0U$Bl)4HXX|o9JSdo#~xvm4^YT*X@>lfk^@v7go}1|Ou}D9B$X?k zfj4?Wkv%O){&_3QEw2T>*uCCfP6(^lkOCL#`bU144suI$<(Vz60niciAMkv?+i3K8 z_Ddkq(xSLCI-k83HH3b;8g_Slbs~02IwGTCo{ZxPB&r4tK*vIe-g@bPeQgU`iL&F#dx3+0(u_+G?+AN%7zBSY} z76hfpTA@&57r9Rox(SPKJC4kRFZKsS1Vn&88Ke}oiz{y=+CH`9WT53&9wqZ7$1HW3*JIwN@q*FP?Fs1 z7Ay)nIsqZMQjOL#Y1QO$CWRU%D1Qun37$6r2ih+Y+F0>+Xd&tTs_(th$cXb0#=3k0 z#0;Di33=fu)4qW{>bVW+=Lt!WN!77kGbaGE%@7H&N7**Ho=2NxBZ*F4tc0Cx6-hN& z?G?=*G4Sb4P2bxCh~4TQG2xR32YVAk*DzHZWXN*i!6Syr9cJ_FiG|!=*G9=Bw*c=w zam;J43#8u80EqhwcpzKn#L!8vHo9%vifu#S>O8TlJNSQSd+%_p+wgxJ-HJ4iHa>=Ou=Ae_ziZQ5I@)@;5?n<4%IkW{{$o2}nJO)O-ll$QEucu$x8Cx+7 zH;U~@hn8Z6~Ut8dNGnbBM8wcn>$t?o+)FTcn zpQU%TE;*6^`^$A%+SxxB>k6D>tr>$9lD9n0Ikta2|42^b6eyCDgS;*;GW6-UOJLnB zh~_rAPU<(i{_*+ESCha#N=b>PY;DP{iNhsFNteP@auv?F|*{ zN|X?lWm*VY*4K8;bh)o{B7}J%WdLGUj@X$rXHs#A^beo+FXmm>bF7p$m5a7~@#|({ zOue&iE5}tHcPz1i_pxjT`RuF3CAT=pLICp zfL5o&;2Crf))2f?|DE5|t8?siN3?uK_s6+z7fstkLG)%ZpIX=2bT)6Kn4@)$8Jzp< zSS{-swtFseuJZSzt)APoo_Ss6&7E!1^801c;7LVIP0GaF%aOEC+RL4Z{4?vyg6tjz zvfP$*p=9s_C`5gtTu?CUq1VI=N4V-(v|bpCT`N0kD)%P6Jo_@N@?ZdJsPznIk;oQk zyRq)VM$8bB!RFL+zMkw{_RF?II@#Jwj@iTvV?joo3iNR*M80!mq^8D)5<(=5xs&FD zJ|!T0-YV0=%hM=;#AQBBn46V-J@sBbc~)IN_$I9D_Lv$u_}Xz8W{L{tY5$EGUCsB* ziv8h0qhbX9B4fGiO1gQzcH%uk$w^{Jse(q z|Ba{Z#SUzcowo<&bazg&bkOpXCF&?y+barbTub!&ErEb~QyFG>PjAMLqnAFlKvc9oUEP7MR|ZbTh@9~M_%V}*NMq)-NPf&kiLaO z7B7KIpOA?``}2-zK$+)8yy96ndD#@VDb)mWsZm)pULPs~3Nl!k-5hTXvLz&9Y=;S< zwcEXu{5ECFai>vKbXH}6U)FsIxw>2CUPmc$8%RO)0J=W%{j4skFW**VE+j=Czs5)y z+t4A%o=bw;hN-yS^x>a`EbPRJ zQs&iIH@PbkBDl9V3t$`s3jTI@11{`eRiyL#7X>~GR<*xe0P`nRcc4JhzH_wgGE8y} zRd!0xC*YAos+;&p0!dx-9+GxeDAVu@oAE(iHpK?!IPL5<$$)c{udyL5bx3mSK{1$^ zG|JE17`ZAHtXLfy!qU$cI~al<%rdhp;O#A( zR5p{KNbi>O;xq2-YG&VZKM6?0WiUYj=*U9#ezY%&Fx4nFR{n02!Qd&9St(|BT zs}zZU@bRWMDsc%fRkO@O19iG|QZA~cFu{5p@le8*Z1%H4Wl2hvbr(B;!Q})jDyTmA zBM5K%dHAWVSm8q`rKM=W(wW-*G!nu`809cOq(xRI+xF$6uPssDrVOAxOpC-PmlUMK z{7H;FzF-9vAbe9=ryu|C->r{H+Qrd}CBHr-hZkQ&FMiN$#EW}zyBNNP(=fRIA=Js_ zU=hsyPGkU;!BiA1!Un@FC%jMGA&2U@kN~ZPT=$cyM42>XbWXw&n>>H=9EsLTSPVb4 z2fGN0l%u=lJ1>fOW>~5uQmBJX{zx+#$zxw%pNO;6ybr-As$1dGgkVlhXL}1QLzA>w z5MT@vKjA2ep5J4VreS))`1&5Go%OjrRSv_?(Siha4pMGnhxn|3Gs%CL++xwLlP2`PR(Yk0s@8d8XG<96#K+1x;4~^c@tQX86O28r1Bn zPrkVEUaxeRufvHKW%i)XVLtgbEFY4Ch1yYa#m8U4iteR*_z@4FFjvNjRonPwxjm=> z4GNIsg<-9xXBSp}wtN9Lg+fDVecofmU<1vdgFgTB{xjc!EuoX^AWGcc0fDCd=6P*b zE5E{t9?WfaYq{xg_w`yhT2rrJ+#+=CytePy+_4>g$1Z$B(QDh)3`jtJ%NvD$g?xmF z?|glE--&a#rE)rJ3Yt7fsmu_az(DDv;^}OXK>lWU;e6-`*X0I5WuZtwFrL{gDEpI9 z6Ucc?z9zvMcp4NYF3Z;CI;%phWA!@6HMznKXeLvf4R(DkO{2v+Pnjzh#9p*=TF1qc z=9aTnm*K5(`7;^U)Lp#9sQg0M;6!AXyaSO(=7njuxADMXkWTkYXhS_HjjzO$yJI^= zq4<;60LcwISZ$#Pl&SzF=0$yZGr{W0#(S5car*fo`);oT896o`pg{E&|@5!V#2z z%$=1uppB;_N7V$JyrAEx%~1ST-#utK^ci6@L#^U#&6f{HgyzJ6_hWHPC)$mQB$0fm zJDTVj=dK;_B`4uoFuMP0bVsI>WM3q3lw1pk&Dp=_{f{XK$ssR3hM~~PG>VdY)=45p z>9vwfpE=5b)jSV?rbsGX{Y4w-CXAxhDk)c2j19#wgs`Q((a_YY1m=nHD?D1Ka3mW8 zj`xH3{SId@6|MD{_E#ObuG?}`{UdZ&4X&>*PC`D1)NdU$cO&+N>8{lL4>fq^umA!> zLElu+OOs=sQCECrC_IgGS?LEPG#_`z4mSW`^Nx1l*weErad`(6>nf=K z4(boK212#z(6A!;H?GeQGg<)8LTnC?y!M}JYdIQ4#9n~9rR>6I3cbPF(U);T}q)rs_mCfnZyWt6VlQ7+XOmRyk@cHzfQs{%JbUZDcy{_|wS@K5SFU zZ@xrU#CF$f2iB|q(A^@tqnC#kSA=Nf0n+d~QFb?DL9*LT%b`nq3euk`_hHed93OsJ zYakPYxXY62#aC;(^q8E{8$u1kZz(Xbh>~IsB5`(2ak&i=F}dT>v>G$5ws6y)xQw;o5}2Sd)(Pbc+;c<_gZ5ralP9ehHlKbowE2bL$8fbS%|**myx7$a(*kIVyi-PMwU;FcmT2O2GX3doE$ znGJvS6qty%iOko2Z$QSdY6_THyRne#J&5C`hf+^d%vvmiKHsn@c6KA>A&J-({|bpc-a2loz_plMMn` z1Jb}V>CC!^Ef7=iOUg5w2IG<0>(Z9>0L1FWJvq5uvATo@2kmE57QrEA*(oksUA?h~ z>eXs1fW3`+*3XDK>}yxx%JM*vgYK~P1MY_K2EoghFnGru0!6;a|aAn6yns$bQ0E%m3aJdxv~vF9R-iV?Z{cA;>kf~ z$zESMzH6~;DZyg>Bo5kPOK+@5(jZS^M`z!d&qyJIs+l+A{Jn4A3#~?G0LNj!Z4Z6+ z21vd4N|bhPVmHrVVN_!`)WK;-fm1ovKWBTH9^vcSIgsoNXRaWfVvhXLM_WeXcz=`~ z?~j`MNoG8kYM_G9YWcG4K5zy^knenGUkGy6!76l6L85u$E0szvRJL-a2wMTZ8gC9x zrjgEeJ6aAVUyuO4(_e8v2XvXjZZe>xKj|U-R1a-`SU@5pyi1${a9Jt|asUK_2jg3C zzU9VG79EJpPx!L@0lVnM3ZAa)ZC0fCRS99pXZ}MUYwacsp>scbqYe_P@r~%^IfQRT z;pI2M6VRpoBFlk)`8O0qB6RtXF%M=6SzD)LavOxOC{*CYukuB%5aUh@bYg*N;fQ${ z7-RMASl;IVJ2z&)b1cKgpG z9v7`ea;1XaZKIo3di~=(P#u{An&KTqCRKY69KEe@w)A11VAR^h_`C4c>j(Ic?C!u_ zJl^T(DIndDVng8otN9eAd)e05*+y(%x>imk!7*O_mk5djV53d;MQ1laZ-dwWGC?^G z@AYSpq97R>a(khJcc9Zb1r?&KkzhKzO{F`s$oPB!=b#RC?c<22AVFII?akjuo0U7o zP$x8^CbR7{{kX?{7@`%P*ki)DU4dYc4F$gz6>U9!sJdlz0A70hUUb^qut#%6$w&hs z*I5TLfKcO>=zH+xp!s4YZ2=UnKLgsB_SOi7$X6Dmp2bh(F`fs`=_JDDAtI_et{lbE zh1W>ZB0w-Wscv|g%M5{Yng!LMPOW(aDrmS8J)7SPRf}ovpb(Y8PmiO9EKA@|remgCVB2X0af>By@x+5(t z14{diJ)9BGZAgNHU|X$Q-^0B3jqM2s*;7)0cc?k$gwP7 z0TK0Us{wU>A1YI(?Cdw4Cf{bxD6&%}he@{pGS~zxoymnerB!locjaY2 zijFT1ogx>if#d%HbfU+YoN*B;O!mYYqE`Z{iVMYvzkn^+fq7p&<&FGq!zjb9$6r&_ z5&%WE_Td?gc6Fo3G#AIwcQXn+MPEIIKnE97-3`_wdgZ*0oHoR^^&J?)^eHWXjm?l_ zE*%0+L{_!=CT8Jb{rM^()0hj5LSML#Jwkon&~C(Q9_5HQ$(Du>ME@UY1yubyTf2cn zLI2IQkja&c-WWjr;LjB8x{-g`5Avg&8$cd(CK{;5>;OUYk-p&hA^PJSZTZub_PHg~ zsH1#wX_aMPBkH$$qZ}$cWo^N<#TB!yHKO>;SUQ9`w1+tOBs3jScM+z6hS%Of`N6TvFpJ)er+BAx_e_~GN^uFRM*z{SXjZ&mLd;MB1}&pY)gKk3LZ7t zoPy7q`8w@uu_VbwsULGOf`ive^BKa_@6GD5S+R>6MYIEy%>V&A)pPsvX#>I=}G!yLzqx zmu3X$OE;7VThlA9piPoRxunDj$0E{npp;2s(?l_m?figvrZSCi&#*H%ertfo6eyzM z*qE>3@6x^3mZX#e9B56@(Aot!AsdK0)9nFpIl|?n@0sxSj`cuapvEiv#B??g)#+Q0 zv*uD_3UVKs!A}}{k|9J;%3LNSdJQ`GkW%o{?%23-enP>JanWQVa#?RTO3DGyQi{;s zi(Ci}E23S@L7@et5Elv?L8u02y+Z0FRie*{pgQC1VilvH`F);%s(l#|p{vRT7X34b zYp-P>Yx#{g&s6^9I3VeBwhoGC$@&R^m|SQ>E1GR9=(P~NKbv6Qp0^x&*t(tTMMIQX zwr6ieU6qZ8)c@+EsZ~J)n6S8Sn$^WZH=JOE=axXQ_)rN;VdNhzRP|&vU(JvUVS-++nc= zw+2{Kix=s{G&jm~-+s&&y#@8E86Y0XLs^+o%IA>K5^M{dv|d4`L-c-QkHF2(Eek)s zcxy|H_m)!-Re~nfI&$|xVdT;5o0rG2z;DO%r zEeS7B!miPY!?+{C`}Ahvh@VYeN$$gR;6|~a<j?0WE9 zD6E+;Sw7^)R3~ZCPT(HN0U|N};?*H|7sOfjZRii3z6l9A_)O}5zLe%Fi_{*NvL@6I zT24FbBey}71|8QYA(wFe4)}QZ@M7uiI5Yf_2R5x%I@B8dF<2SiJ2_LmOLxl|g51TP z##Qi$x$b zL!^?Dch8(S)Cne73uOuCg}ZI8t1{nmyhC|!gC`RPr?5Mz$#Od`i!D6 zo|b|M5;Lh2K$zTFxYRfN!h*HK4l62`I{d)&yQcL1DW#?4~8{CeKH zYvw16p_3&9RJjTLu^zmYTUA7{1^#@+?(;`H1BOt!A-hBw&SxAbzEHl$4yo~$Rt$`BzULSE`q%&B3NEjGUaXNyGr?4ClOfL$Rm6=ozBhUW}XX`$^HReZ| zh-fGZBKoY3<0?Ji_5IC6woQRQCW0{j=8(wDLfl8l9X%o8?4ZT%K<+TLV)#Y}A_K#J z1k8B$J7{2GAm)eYBoCz&xc#(O^VR|PX#Stl=~Wh zUwTo(0i~I^^S^%vU*BCwX=xOIEBUDO4yna^xZFOSP%}#YKZ24PYGtq;*5EB|48dwD zn56Xx8#TvAkKLabyMS4O*e}}uP}i;v>cBA^!-J7}4x~s7m*8{;z~ANBR8z}QvRD+b zjb^pMr<77dg>2fC1iKL`G!rb+7p2VUr%RJ67oXHMzrH*RMc@Z8fgaRZT=fdR@! zJUo?~uB^xDffr&8(#yX&qaeI62P&I9gv|s0^+YkS42_tIAPDU&fNabtyP!V<(e|1_ zIl+>$X&(u-0?2vRS6O@mF1eBTe{Osy%Ntbajzm7qc+0!~e5lD+U(E3cfYMPn@`QUG zMtozQ`Cw>BMHIO6g8u*t?gRAlx1f9*f_kl^z?zd!YQr%0+jc!W8UeYK1C)bZ$t)_9HK3jpMxd`-w zJ~=Chay0tq1^A!BrDFT}xuvDU7HOpydV{c z?zaF=7EtYd;!*{bsezN}1Z1SIt|EHIUd^ zxJ+9Bp6GqZc+*r(vu00Obupib#YLi7aFA#fe4x_@r(HUCeHCHwOSqki%wwFTEnwy6 zNEgb!LZ!{Twc;ulu0GX!`{}{xBYw^c)rXZosd@z&2sZ{vHv<>go#L5n?=vpo?dAg0 zRloTxgQtK}sx!nEW?>8uBpvQeO(gu5N2_`42spMrGFHPE99 zY*-20xkmO0@7A^!Ur~l}1c0(LN^Hs0y`@0m(EbsnR5MH0wA*--QB}a@SMi|mzB+5` zXnJi*htccUyFPP^sL7wZ*o(_T(N)glqt&tghdecBp zq7B0J8F0Px5HoXd7&L#Oa5{u;efJodBR?sDwqs_;(`?hC6*X7&pqxlP#XJ8s1S|Um zzR#Q9qNy5=xQE5JPFVnwhkw0P%i+hf)s|$6LvH_&yHDV(|IcJn6u7(HU1;VI#Xzha(Vu>S48g@6@rh(=eNw!l6iAZ zyR~Ks(eMqwiTUKr>-T>Qf|I^dJN;cXCcq}1+2$1;WIC>(Oovr_-QWZQNAdPJngUgm zS*z6SwIf<1g{D0fwJ(5>u}FxNr)G#a+~|}BuLzZ}R3jINFHh4!!9Ma^C(hsX@b>0% zmpD-sU8AWVkQdK>}o75NonfdA7~ zb@tTs&d0K%qef0~`xrsNMPZWQ>?@`ah|v6b?}@O0H0GXjp^^>J!fopfh|s$)Fb!3F z*MgRz9a4w2JUmnb6`LAWnq*Fx>-YGY0uM#DY%RgOcqT&~rF-A& z*(}mV6*Q9UESX~ zjW|V+RK;(9}E^c&n#zyM^kQj@weaHD7FsNR_eRM;Zq?whC^o=Ti#`&)- zq?a)y9!%vq!WijVoPGtq2vpE|Qm#&Ce-)w_cTgTAn83Rxk#6~XgFr8n4!o0AlB_(& z5gnjC%>&99dl7KY6ml|spsB;6&W-H`KanU=j2&OZ(%W8JrlCUBEvAd$s1WT42t*%&tD)iCKKa2DsPsfb}i3{}Ws;oSu*bra_V=%Qlx8N55 zwZ$G{zYgKzQf??ffZRVKB!$KEK)>8TH_&ePGaXz{B81CjiCFvBiU%hm4MY-)sI zKqS5kJcvBSNFON^$CXLwa8b>GeYlRNdPG@dsX7!U0q!>y(cE))DWdM^*Dz$GpFfh9 z;qM_RXnniqat_Xh#Al)*!xOJN{W?~Ewg^KVA^ge%DzpPB5@7t&EESFLq@V4Q_?p3= z!80r5L>dUgqH%}}fs2ku!8unEfHhYrPD=pY=Jy z4O#xn-P9&fB54{G1pFd}c3bC9#zgp`GM-8HExNO)^Q zRJ~tT4nBmI3sfIqLjL8dj@#UrOLf^a9VZP?@H{ei=X{4 z45%v|H*{D}3w&tR=Tn4-kbS!JcbDBkX9#0 za}H&);QokUpJA{MyCN%vkH5lntv%@>xY&)qz=Z8_M+|6C^rKHd1CBxtRM7UF@qZ%E z@v25&aU|_tO8M{S%yPni-_w2p=dF^(Iz3k&L}TR>ac_3fc3F-ZMEaro=Z&BkfEZ@0An6EBT34!A%rzJ*i~G79X<(ce!W$nFW4r?h<=8okk>V zRQOg`x$o85{K?~kfS6(|6XiemiSHjy9V`p3WT7tp5>Z&gu!5t?p;9o+<(OM(_o*3b zdk|Ho8DD3+G)o>1@l(#|9cR<~^+u=u3W=`*%4Pv~Xb*Ig_ZME7MC~l@ds!Ed7ylaj z#EtH}X_~pG$Wk6lnjeF>(~xEI^La9@WyLs&e%7CFea|*#n>su&7Ww-=9HMDA4sIZA zBf#w%XSf^t4y1^bOg5BbM1V@D&zK>=-!OhF8(BAwAEL zCPUrp=!z8c%g?m&nox;kG-|7E-j7R+Bu^&RJWsGb9{_*bS?`Go)$i$8D|&Pj>kIt~)O{eMy2@iM!B zQ=|f`g_;%7dLC4|y);^5abi}NTpy@sUZ~!}0?vsB0V(>`Y1V%>S@i-Z(^ zWTrgvXqwo${TsLp%o7YSXtqdS4vag~dGz$( zlR^Lwp{iTuk?Y?t?xUxU_ojwwCA_>tRw~FzxUV74G}Q7!r)cbzKa9zM6q?)DPnA7q zJk6BFp@WhCmi6Vjjz9T7?)_K0kjEf1Rva}16O4aau~qx1xDLB*kO;(=?9 zVMD42f!>xJLrxQgd#c5EBS0DBXT^=y*9Yj?;Np_+m-2M+^)_v0Ri6(DE?N9e+`bl$ zy{9Xxyv0QtqlI%yj_f^s(=WEPLnZ(B$cWWmdJ=)k-L*L${T~?RssGu2l>cfT$FNRu z8VPHjC$s{>D&H_o8gaP_>_pyi)`d|@S8{g~+iS*x__4$<;e|AWQPc$SM5yLra&u)| zE4@TMU4AJVi>bOq3J5-q-__X@;HCKPm{hhVzjYa>a%VlXUKU#WMz!7?jeupw9;xRivfK zeRh+29~iFU*DBao>RsLY*5e)@f@Gc*T2nUjo;|z$^CxkdW z#>Vq9X@N%Pl)aut! z81v|DVG0AOnB0#lGO2aH5Hm*G-D1sD#IEqpQ*J=%2q>t|P6bb~7O~wX5#^>BcH*X5 z`0W3c!8J2shl}jwXIBzg@>-5<)=FmAL!y5VKr60j3;||0q-EN4lzT6MlD_%FOe~v&`5#w;p*(~ zh#hi~E|wA}=X{|;-y|I>vj3$>ED4r`TBdxHP@c-gIoglp;=1Do+r4Kbm$F$ek=Hf_ zrQwzb$Z)`jA&!ZQPxtcg6?Z~e-1vhe!FAgE)RGVG4LW@y)XjtG>g>uE}Y@}MMXXfK;Q>Qhm8C+hLaiSRtlg0v13Y^&@B=lgq6e36b#+V&fU zeET1*xUx5CkxLN}V#sALm23K4Di69ckFGppx~*1j_@S+(QU=NwQbMSFdi9YZ1dbMEYE^qH8<8TFk#dF#i%7N_B!;23_R_ zNvzo2tM@VFQsw-FMg2v6M-&)FxOAP4V13QWrI-}UeGaA45=As^`QI^E)UqLTCDFeW z=QO`AX{vYHlnD)nTC3ctd>JT0-T^lWb+=FcN%&XK{jw>`P}j{XIV>}ktN3j%3Dc{1 zH_XND^5kyQ@xvx?Klk^04AaDywttNZlfmfSq}GV?t*NI8{WDCfvN@9A0XL})BCF5z z$1=V|i%kAH^5_ZyVKGs~xoSp|^|+|jnYq&Fw-I$!Td!40A5`xMzg874JCl2y5EFRh zLr4Po0&A~IupbOBmz9ad{*7KJTg9Q}sobx1wIoccOoXmK+kO_2J{=9H^dorviXJO} z5td|&sBy0tB2e#vX3syiW5T#`Vcb)G9I8;pC~BhL5D~FsmR?AVmkuSQU*@DHFdKO* zNxk21@BT;Jy;*2I49Ma0HyA6__d55ygztZ+dFfx=DlI8ib%pI=(s1WYu;2JK z|9+xJF7NKl0oX#Rcs9Zd$7MgqIuWXtHbvb2$TpmGX7ec{)kb)gQ(Vm>F0_cD|6^Uh z8VsU;iX>8#nJ1r1>}HuELqd7I+=piW-b?HUc>ODy+A<8Xei;%mEgL;yTGnRx)8w6T zaT>mFPN*ZipHQVoylNX{+1;%iV%m04;q(snoeH00v}nZ8{K&)ERL}l?_bXQNXk;FL zQxC<$3-N~73zt&pjq|1B>2NEo=lHe<^3t^}WVw{^#0MsFJIBzzZCxS;8kiHj#Ej(k zm)Id2$Jw357t#W2H#>Rtd0xw$e0A*9s4FY^>1n}dc41m@knTRls6>c9%V&Xgx59iT zVqn+sW2qH@DH0In6CWp*-2OFIBuO3ERe!eKNd?0Zw)W!kHdA%U*?Fs~tj?HjB6sHf zHx4^6IF3^AKm6Iutlh$nR+_>c*UoZ-C+t3d;*ULOOzw0f;p@+~D*PRJ{I@6AnQAZQ zM5n#+5Zh*S{SA_o-t>fk$b#g1ty=m%vHVPWo2L=lXb?4IAc&Ur47gb>vD|*buzcez zX2~@)F0@EP-DWZzBQ>6^Q()0CF)sdusbIsmhguz{Ak^#U_hnpy&yso&i|yHapju5K z@DChQEMdNRomh5~RNC!ua#;q_C)zd!=DQLMSL*dZBIKV0za$`3Avwf| z&LyZEEh!>~3#U0vIQ5J6KjUhIGuh1XTlW%^qrdRe{p&~K)y#48@E%=@87S-Jnq`?l1X8RIw%6^ zC8zRx(>+=B`rYTzfyjm#)DF!YLNZT4ScTcTqq5+5jw`T>E04(Vt3T=E(UzU)0I58^ z5^ELQ`iXj?BN9E%jGKFNlADLr>J!GD&Fu;EIi6opbtH_gLsSRcM)}Ez zwDt>~SODOas~RAhu7?WDoPe)YhrkOmKYG{VU+_v4hCX`Ob+l;5_868Vf}k2B2|T`C z?ynph*lrhX)=p%8Mp}%;!J0!3t~?@e_aP>6;GH9|!_6E9p+UHF^PA%Dk;88FkJ$NA zEp7~Pu>U+?;V%|BL2MzQRQK9#)otSS3x=z}M|9`VsT5u`eQ;e>1muwx5;8Jh8GZCT5H{NKVuhJANZt4;~!ohCnE?^@tbWGSfu?PB&$*w4=qc7{j z0tBwUeYpDu&LlU|$MpucmwxWl)Ooa2l)vcxNV~jwyz{hryS7yYM|_MH6zGC$1zPmNA`Or(!QL_|_0Cl#`)5{b>mI6L<+=ydt99h6;% zBj$DnKApLOS^f902wAwJYduLkTn02vAvK=0OjA?Z1~n*tfc~aJ%OOiqjH>k%(z8Ud z?ZtldNLcdVC3y@bI8%#Ww8VUyXU~N@{yXPu6v|1QmX;2BYO37d6N_#aoCV-=i*|)J z?tJUMskDdR<5UJYXS|z-Nrz7l|B;w?}VPODtUJZ ziD)>8=-pguAGGvHz(%#DWL6<{pn+~HFo7GRUYo<}`A0%l_j1>d%YtatF=7^LXY)YI zQNmsYr{oy#YB&-ma&;8ZnSAyjb<&G-l72Ydsl=Vzd}&o;SiFij`k({gl8@&B&4Fkm z7Kt~ZJ%+@gZ<5Rev@rhjgZ5jK- z0O}Xl1Nr3I40(`t0(TnBXBRoDgi9t^UK7B1aY$qBKaH0Q3DN%lzxTNRVG!W&VtASE;FePYznr5vDo9H3b9l8o zUAcE0I+^&uML$Yu60QCsUO=heml;$1{duz=esyN6$6a;6BY`vX7Bx<8)incAwiFYa z`-fYt7_n2fM8(Lh$BBkDGk(I#BIi|=kU4Uz>dd?6zP5zCRL*q#+5BuLdhp*80p%qU zYbtNPWnY{^i?C-q0^**H->*}f1y9%hDT+LzJFcRX3+>utV<9<&O@PP^zUD`azReVg zoay74-wP}UpT{i{{+)!I)H*0OuP$Vh!Kd)TDtYVDl8BZU>Df=c070PEa-%Skp1axe z97)Bi5UKC$hglr5r;Z4&nTih;fdF+t1TiZ!og#e-GfMDF^ZlRTRELJy;Xmz;bS}nx zUh0QZ${v5{OBt)tQ|TVuRC(&OKFH|encnYQlPYHA+hft|rxn8SXT+)ON=BS>PPTRZ z)T4yn`m(a;RdI=%W8>my)iGiX{*&qMII)yU1C^TLRaExruLpPjSFzjikIYQ_3eVH) zm5*?rs>1Hed!X3!xdlLA`%d8!q?Z7a(G&B zA-1hb_D()=DvQHIxq(x%mENi@VpPJDtvq_=SI?gYeslpW+xhT$bqgCvl5{_oJ*f<@Zk;+7fFL{<^W4^AzyMx|%4A`u1AK zqsO&`4b!y6V#az;*LpMKW@Kc(@tMANmbkmaSY~;0)mtNW@0#P@T7PZH_3hqmm!{*S z?S(E&&wdQv8X1vpAGti^diLaC+>ISQqQNr8LnM+bt_d(DzT-Q1sVx6UKUWq^&1ju(F@u*%_ls@f&K{@z3jWna^r5 z&>GkoOyBXppTK9z`q`f`{0NbMlGt*7x5RJGxM?broV(TcN{iI%r?#ziB5aZe*0!@p zms3{LCcdX^rTAF~^7OYGogc$;H?4S|YBICbi>gW)iwF=fEFMZI+tS>q3?3OAD7nut z5_~_O-@jrhA-`8>Z-L-gWR+&!h2Ds&+v^#jw0#4?rVL}tDT9{Ve~#s8`F|_ua5>Kw zKHFeQ#K2O)fGY%Y)uMysz`B)2074a5}!zkuUbh`r|A0l$!cA;Sa-qfd;!Q_)cUW9 z&N4|NhILJdY>>rN^Q!b(QvvJbqz0!s=6U+TmsuuD`jm|GSpy^#uEOI*vo4|5 zIo6|tag!a5{%V^=vOC)(4kE6H|GL2^K*9y6`%iyC5H|` z)$N|mlR|1=G+0#=LZaAV+cMBKfQ?Q?__txtHs<@pLY@!uO;f;71>b<5dy2*-;dB)1 zX{qucPD}P@y*Oi1bLDAFsJ2_s%d7U1zFFBnPD{O7O7!S^l# z*mDQqZ^LdpJ9X4Oo`8emB847$WjEOav&K({Q*a7R>O5?fSJ!(aa=0vySC>I9=P5#E z=D@(Wy6Bys2&%?^+nJ$=5wtq;D8rQ{By;{c40$SUK_rZORgl%kVqh?>6xa5k!Nfm+ z!91Hr{?R>CBelhkJ~V7>uSVs*de4_#Q=dJuZ~M(S|5==hpCV&*G|Mc*4oKsaJ|rYC zVwKbP@v%o`LXvbQ*>?%gSE>_JRhb$U52}P_vgRx{CSxz#30f|qbPf|a`G?15hp3wr z;KT3^`7=`G>OGm3Jr6(tq3<~Qs#fK4?+Izk?Zqvnxrp4PO72#=4E_8lMf}2@Z;Vw;td4LLu64T(-JR@nsW* zRJ;$7fM0{b!-NL>QRbs!8Zu@NE^7k62P3vV{m6ul+~^nU8@0lFTz)mq9a5(-^K^Cq(LYv9R$DuU0B3`K&z zH5G4ld5kmXv%iSh^ts3gmLz4wXnOX$mQ&HpmHggT@R=bE8NG~Yr3#0>LDU<) zdI(Cva(MvKN-D5fOB|+)y0hyjxC2|eScC6U{&P5sf3;MJCvA2aufSFDM{DW5W4uo$0A8BDsg)xL z90S@Yz?|;T$5a8QMw?ksyJXn8VYo^7I#jabH-IHvban^Ykgp+z?A1-UJV_1Mq+7uHcVI!wz&6F?6VDLK!;=x(wg8_LUBVZHWpHc3HYC zpRfhsY)8AhTk>-j+Oyi?0uyi4t|7-0w9G#60_lEY>NNi4bqu?`hxcY*1BW2 z&JB6;o=(w<$aI}!tUyfx{7_xidT%|jhrvna8Lpx!IH0{kqN%V)+mq`ezJ5|l%-#$<8^z-!;^hBmvSjNh( z^KK+xeaC5qt~9O3c3_JzGoBo=%`DES^>7tVpe|}Al;?3>J`3kQGq5g0KS20C(39}2 z!pl;Cw@B0pr zZL6=OUMD0Z7?wh}zrwA2cH0o)=jaY&w#R@vww$bcpWkP#kOxFv(}<}5M875F^FJ`- zuK7PpVP)44jYH-RjgHytK~sSz*xCRJH2^P0#}3L)mBEe3A$JeO!okt3qsx$S92mWSb@ldxA?UF6 z6PZf!5`2=f(L+YPBC=Eni1+J*f(F$If`LiH4W8;5PaT&4eENP)KyTsNqDHad; zosVPk-H05Br;#i$b=u*_^KR+ns)GXGAvMz;C||BYN8z(6)0`OEg=;{g{jQ+bs)}Cpu|+4;y!ZJdq zZ;or;Y$Eecov)*syl?4}G&PZ(;oOC83V7T^R$`KWs?|G4S>#=hvT-qO>lKsO^od`g z)@D2z3{mEDQQ0`RgPb$lZOhMmMSXt0msyAUtvSwhP*HO;#=2mJqp$q_68=<%p~vpt zQ$m$q;_-Ml%{>gptw0NYQ~Y(w_QtgBnbH=fBcv*$S8P1lLYrULA!bf5!Pb}v4=L+>Detp{~cD1SpQ=elF(nE`;*S;5vgoR!2Tna)Uwi_iE}=YdOvg@a0Q_F zmnng4Kf&7MVc$ckd)1d>1@~j$e*&_Usg3aFFHuir%R)t50P5&OZ<&3!^z3>)qfx{k z{zt(>we7a<#55@m30e<--WfpcOlfxlgU1d#=+l#GCLSD+jW!Wp`jpO5$GGzwd-J~K zHT6{+%iv%a6(ZprhlCfWTlePNvh+}@JYjq7G54J?!P@F{y@L+*8v{mhwE&XVnuHE9 z+Y_YvCF?I9N_s|M&KdrJE?C*FiW8OU+&OCHBT9|B6en^r+NC#oPE=?-R*^UX9aDMf zZ^Rd62HJBqdQQ+RRc};0z~-MEI@KfibPeZjP;11i6|&4zEl}rmCFIpP!E1bunKz|B zb7ONCBJ}qV*?yJWPa(LSZHAEvHKOGH#<);jvkZjvt&>$zTm?yo2G}Hy2zQ4C$<0&7 zctuCo)s&Wm#-VE+3?I+0O?5Pt$c36pknI)w^5XD&hcM>J9lPdmY=-QhVBLFN|VErLGHG!ozxln=a^f?+=Dl zq}S`;UTwACX5$)K9G6MWzswW2e3sXws#v^a+ZmYnQ(g3l7>?bcrYjn4kiN#<6qT0z z(H9K8-s&f^?GN8vJwuO;v6-I`wKK~Tb7Z~~dn)HAp?_|HR&|hJAJ#(w-U`WynKwj~0Y=8B#P=dpgAAZ5=p_PmaTXXhhX><>pFUGj| ziLZVCNLNvJxrEf~3~>3$CEF(K731|vXc<=W(1mWRN2htcuu~&SqdFI={OpUZOWz5; zw&p+ZN{ypJT4q*OmRV9wX2BWkcgIpFTr+UnO6gYJ)&)xC-xIGFEm`k)lb$W{3|Yl) zyJ5%A@8>b34%FR0aSKYY5%$$HSf0ti0QGG~D!v*vHN*3S{yxn6d2r5g3A%}WBMa7I z)0l1Od{|$Nt^$1TWdyIc*gpTVe&0*6AAbGm?R94O_1!GzQ1CN+EJ#5-YHyanNEGY5 zR+t0Zy%W&D!JXF_CDjL95kC~qBv|(JOIS8mY;5d_h=uv1~4dd5$=QoTvJR_#w}`?*soujSATmj$jZSktm# zw(LTXhj7K)7&mBE>};v9ml)Ak;NI#HZf-Uerw6Y6Eyz{Wr>B8TKebF3xo+C@%c&8A|z@n%~eqrv12S=gJgkoWkZef5!Kc^2|d4 z>%NMd51?7=cUtp;qc)2}BGDd~lFK(rJ=5#x7G5ePyyEbEfL$A$%hIa7y_@-U$JZK{ zw*CI5Ewf7O$yZfW%D>5ZcnDEWeO(V|PWDmgbE{HJ=c@P$)$O#f^FW{Me5B}Jkn7?; z8rqu=19n)%5BrcZOf^)njMYpmak7YSdG{^WnL)NH;=m1d;?Q4X7HSoq5o4kKh9^kh z)!+Smpj)7@TEE_KcpNf&nY?#CdkmdEQY0FF>iXQU0nHA5DTC9`lgs%o!9$<>_BuDm zU+iew(#xo?$br_aYty&I6kl7fhrJurHq9^?$dvF3^6^%EuX61lhIV zzL)<;+k1y&{l9ELiQ$+S;|gv$;>DsSy|cc z^LbtM{*L>&@B6rq-|znKbG$#gUa!}4oa1pm&c_*GT$r~1d7-RsU#qdKvx&m($URA$ zyAoO_Xj{HtzwL#EiXeZMfpCIy+4-TabOSR46X>eRRIVco;{nBTtEh!BnAs66qlK54 znHDt_URGJPYb5|M+llXIX_KGbpe7dW+;SoQ0HoTy7{P6)7!!eUBqHoS4DAH<%g}qT zgWT35na;;$yPoFJq-}E=j@5jg1JRliE{dmxhpd5(wx(BF_WqQec2(56AJlV2CLVWA zo3+Qjwh2EalDCYKtyBoa+aUtKM&HF&WuNf7i-lQBQStt$?su|i(o5gJM;&+{rXKCG zTNM~dyR^ZnB@LwZ)h{{b(boEGbTK~3qIAG~jYGp&SQ5-o^CcB<2ZZ+}Rc?X7rm}cb z0{!LbEE9?QMc2ad8HnB0zL$Mrq75zHZS7Gin482D^!mZ zjs~yILQGSsmh@ndm)vsWZOpibRzT9*mWGYDWaafTIZ^Jr3FnjCcOqe4Lk{Nkjxs){ za7$Q#-^va&ldkJ!k&BK0I&Aa8KB;+-*CpRun5Er!At3g3`sTq92wmnn7~_>U$yXDz zG#gpdI*hA!)&#rRV<`;1jn@NP&b`>Q-X2J?{4z+5GYR~FUcDNT_%+M1Kb;v4gfa%& zxaUU+49Jw0*)9ar?Q8I?o!+V`JR}L_)POZM$#V}&!wP-?3wzJVk$a&*QbnN_MmA;F z%tI8CX_HJSSr)BkQrpTBs$CUFCJ6Jl%r6z)5@8O+bTojRSB49O9kb3&1*~}QL6Sm0 zs^dX$D1n~KVb1a*Nl1s1Me?Bms+=W;C3*8^XN{1p71)*HTEG=Ver}t#62-;o8sv)q zC<{3$#O+=u;rH$s#4lfOZpqGaCPZu|0CB&TWqq&y@kc8VcPy?2ZGt@=o?8bQsxxgt zQmw#XYnqMZh*$IPM6-z8M8g-VyHmH>G6Fm2zY}}e)^Np9D>EctwK*X12ffz>OiEGW zJ@8|r#JvKJ8X6+xVrrM!u~3raLV8l5XHQ`*#k6JqFYgG#+D1k11XXUQ$;f36dfm~S z_;`bF;U~35*)O#xzq;n_nf<7?klHo}*z8)~D5m%a-BWy{Sy$U%Ez;c(rD zyY*t!(M5ME`jgM;dp|RpEHgJ;6q24I)Y#%&zE_a=0AN}9YuuDLEP}nkjFxYd?g8KK zE+&U=R6`F}0RF|Fc4c(`TWZmv+dP$k0B0@up90QH1c|e~M}=7qs%{6$E`axN8N2)R z6wnu#$A8Fo(0Wshv#~fyDfj`fc@&>6x#MF;1sQq}(-8m}crh_wvccbrmF6ygH&=SO z78?=xe*%Fmum$gIFHA+NJ7CL3i_EB0Wh? zfEUq4I3xcATPGOL1Q;>qOOI|A79??SKS=wUsZlxPNS!P_if;|zkN+iyG{(ap z^C^3_-+~$sB*c9%26{GvQiQt}*75G*%~OCopNIEPd8?iQ=rKH?nv`k0?UGofwlXf5 z;HwOdBP&4CnBl7^xn9Tx|i?L{1ONKt|r)bE;u;U$EIn<#l3}1z{QCZ%m=E*%kaw7nV zpUx@AvD0LWV4Yn0jH6E5p&fqM+@->Q#F>1;W$}s_lu+~GB$cg8&MTnMR>=SmI-7~r z4o}hXO_*a_BsypwvwCU3%m&_y%b~y5nSj(83KZpUpI+?4M#CE_jzeC68uUB|3_TE?n$v%_7^_OOVsdXb8LfhcJ?m82cid5B|l_*Ey7HSo1fQ zR#%hZn5;~sittJiPlkv8gUFYhHdM_93#(cuXN-1^5k7b2@}Jn82F|FtQ8{lPw%}+X zQ0BygpCW(lMkXn&Vq@Nc7R&q;N^EU5P*w*ha4MRKoIq(nawbF2b$;aF<%=WK<7{ts zrJvt}BP0!{V(O=>0hUDbI0EV);vZ}l-r>;nCq%&2BpYW7UX;@g0*R|!paPZm&camr z;wK>Gg%RrD%>cmliw(Wr*aoNq^g)eT8=4L=fa=@nJXMduLU~$b zOykOT&nHNfGAxDBo!3Gl4Asr)J)4wpDbsf2FBNiLev%2(56NnNMnoJ3isY3_=$H>F znr^@ERQu~*^tYlA?SPH9jM^Y8W0ZSpjjI|qI645&S{i%ZZvtra@ROx($-iM0@cC{n z5S)A1YMfx9dn2H+96WKpwBOcDO=rsatEeSu7C||Cx!R{~p$l9aruVs6x`)@8 zmycv>rM9?cJqqs2_P}|KIr+jDdp*nGxKb8RV@n`7L^4G_wy48QY=Ciz4n|stQxczC z2XU}Ga3}JnFWf;c?Xb*u;1+Ry0#x2L7n<34swn{k+}i987|ngo=(`kjO3X237W9-2 zK&EDOT4<~rcwTvkAH3b-Z!}UkR7ApzZ~Owv{3odHIqGi+&frS7K(faLJ+Y6g7|CXQ28bJaYYarGFC4-q35UB;Z!|mqjz|g zq>0Z^E%_5vYOcEc=!LF}bP%sg2X$yxcfi7OfX;3F)U8MPsjUlGg@h#2Q~VmdsNPs;d3cB@wW)9EaU=C}6Ct+xV!-+U zY&ro0emobd2lVu!Ix?%NognmMo94IvgA${3z`aYPKLQ&t~Yi#yEO-vec%@FO>UrXAr(Da0P;^ir*d%7hT0)`;wIZm0CS9= zgH9y(c@5|?mI{WfL@}6-INaQ=3-rjcEjtsRnRrbQN?lOkP>=@YsJw(nNrBmM4o@I5 zIYHx4-hMiNM)Mr`N7T)v!s`#?IZqR(b+WK}evn*IV~<1#fdMo>yXUa8MyC4Aa*OzF zxz**VWok5KBmGQzwOWO=m1WOVQHYoqw|v6?Bolu`;P0Op_R5L zms+Rvk;ZehYSE!%r?ijdLvGkCx6oP%Wo{j1fKPAxi3t#E0jwcW3G9|1-D-`!?V?=} z%(}Zl1g&?Gje8sP1hi4a%V5lw@GR-ly?Y3&vqk78Or}Y%pX>1DNXThxWyqDli`1MD zRDT+9|A&y&Dh6Eav)EEK0~1>2wwNs=4b>VU*bG%;P^yl+(x^CfS>%)#=#^HNXd4t zU5Fibfm}Kl3T+p5XZ0Gi&~(hT7LIWni)2f@$WCH5*MpnX)tL6Mktm6~YYSft?;8pI zejH*PzHmmN@tfYHw=fv&u8<8x!35Xf1?KM8h|y>})?i0=SYrwkk=-aajj@iC8IOuJ zS|GKE2f01NNcE^sXUUg05`-WN&s_#N!=gKU-m%rUgOA{8TnXvF0k-X}+Xj~6e;{ZB z{qFpu`YdqY(1cXJ^<~jt&rNJ-`Z1i7vq>HIn@*`#?I>0+mvDew=AgXa0#&89|(XN>DY%pK17_u@?fh;H0nkm zFnd!ceXCO823{m)6MFkKm;ItQy-`oUt#IelN@(89;#4g1U-}kPGMriEQmqr8V6A+0 z^5l}xuazi8hTN^SpVamou1jv@6X9oIp>Bx#{&9uA;5IiriX3fFjBu;+ey*N5L~1No zu|_XA%0OtgS2TP^IiD56bTQHVcx<8q3FJ!IPMs2Yg@ zSD1ljXI28T-C@{=@<8RpDJ}km=Q$|!y%rGxPYXq(cH!NWzD=)PbE0~H-hX(a`n zF^_Gj9(wwp}b{EmAhb(UvK?x*e^7tSkl(_B4>dWC%7Mp zTXE});AW0VDvIH%t>1E}`%ag@UJ@=CbzBZ5U-AX~1681UgStv?yv2Z+&I`L=s3ZqV zV=9x-k#N$)>jry<)nxoP7n=Byq|=K4bKF8^!Uyp6wZh)4d6)Zz(8Hp4Ipv(DMy;gT z`=bg=GfsH>Hn8*6Nu+5w^FxYiq^X89FUuH~jG%&M3pG3bJM<22uo^^);w*>z9*{37 z2|HF#okTUapJcp;>D?YX4uZT=+(`WuM+g3HizwnGc8p*e74vwMYUx7VZxbnlX*lA& z^GL<95A0IDdC@b-s*FHA#l#jcj4k24JF={B9>UGNaP-HZbqE@VR$On{VWD2cx#NF+ zl~WAyRLV^s8SoIhqItf3pm5j+1i9y;S0k@SpRs~jFFB){Y71rTp2Omh=40R$O$T1~ zo{@mtGk;Et1ec!5!G5PJ?U@3pg~)4p-k)wGU=Z>3!z437R z`?4<19UCcRIKG4E>v`g?N`4RZ<4sRi1FxNl+2N6Yk zKBr%!6%C9Cvq-5I;1#|;qFCb!g7;*fF@FL%Pc;SPJg@mP-e8yNJOrz4%8HdZW+s^= zUqz4(SqTe-=2QwKDBUiARP$xK@(tX=YD21-M=4261SuYW!;XuH{Xj{hqt`=O7cj|e zR;q3AXR=1{xwobUaBU?)&Tvwk(mE(%GF&J^+iSIOR3edz*!(mdPD`$br1&pQ?^1^1a@guI1szyIAl8Bz)9~zBe5lK z8B7+y$A0iQMs*P=@F?B9HQkHtlL(TBwk&ZHHsWeJ8s_0^1=*^7I{lG!;Br-&AFhm#HZtOq))|-?!^NIQ9r}h?@F93_@o` zT7Z-*15C5fy0`-_Aq2C?d2Gt<_XI@WpxMWDzU_t_6CWJ6o@h%M7HH%1U&}Ym1*|9^ zl!x*`#?th#6hdC1V1FM2`ea4?wM!RDNV)3ZS9<~o4`9DlNeO#Opk*r?+EOs-a(6ILCwyK5O}I|(>=U0*Ct!bq zyG4aLlTS`k1tu@Z)$%pAyDXs>-Jix2NeA)BRy;t@OoNqEPve#ZcU z!1;Bm;tp#~Ep-0bRjbzsKs=Uj4LCyi0Aycv|M|JlG7bg}Dl?e&2-?W!!dj7ZmCWz0;_P*qFg^5bLz#2x#C9AB^F;3&|Sj+3hG`cYZE~ zCG)UaA^H%)jnzfG$Ab=IK;ol#@&Kw2cy5iMw7 zeb{zHEo7}D6wCY0P_2xIQ)Z)F&R9LC?0BCaFLXqi=T+;})rRdaix|YuKtqGZTl)Hi ze5`<>p^e1iYd{?#Z&7u;eBvH$UU9>ofhj)6$_ASAY6~=_9=s2|8ZyQ!N46mt(1Tqo zXW(PYxMw~H%&Rz~c$6z)!sepNEL^O)4W$!I_jrlexM87{joYyojH?`7d!I2Jpztfl zCh8{~hF7B(-{qzi23moRvX3+!4TQZ*N9oR$F!%6PjXZ}s;x+Kxx4DVkb~>P1oB`oH zr*XW|=QT)5e^z@;iq30Dt;GiD>>4Q$A}Lo}kXF)xj>VbP^G;0HuL6E^`Eolh_PP#0 zS2ypG)IQX7$zQ*5(;fEo^TJ)de?c5$RwmaWG!MiUfPPJwe;!~f9YbwN&<(_wLNriF zJ#BmJ@k5C1xT~ANsG7hSfK`GHMBdrW|KY57fuZ`W+V_!P0xr}1G|Cmt_hMoh-rP$@HAr@H%*iYu+MpS7Y{6>6X=+TCtL9^O7Yd5hEGKWBEn2_?yA&AaE`@@ zcO=#3vgDd1qsXBi-%(-ipJEwmblM%7Wqx*)qUw{LDU8&K>#Qz^p%=6L4Vzs~7=&4v zcGQZmovb#W1HY?hDIaO{oxRG{N{&et3^GV(IH9Ww?J)?y@ci%ig|I_O=Y7_dueIQy zZBe|F8*0Sijs?IKFQn##8X2%tFq;Bd3dvMz3H^>Ry3dnwJL23|B6mPfW|X6090$3< zB&eZfPRnk;6YzglMQogOf()9y%v|3zQsUY%4X~eI2aXsf&TB|TS^a9YiONSOV@UX2 zK|h3`t5a73_Eyr?EOhpC8IYxf<>>W%drI6bXFub~w9sNih&&LSstsT`Kz^F?$8w0n z^Ffi)^zd17Osi4~tb;^P4%47NP}qB`oDx3C!GF13Y46QKA=ChB2 z&P;v_Y=9c850=5J&gs7+y3cO*b&uewey@%q69avn-ebNHxUM>MPI*iTHI;_d+f=lA zfbM;cdW8Mw808cTeuip`Ezq9Kd7FkJ(fiUa`iY(ISXb5(EkIy%7NGo3E2)LAzCPM= zsGhGGAvX@rK|{R?^2~2qRbQ$J6OUG{SF6kCSt7ve00YZ+M~e=*J*b%h?6jm~&4xJ8 zGtWa~cNzwuZuxDBsYv&wqlKn*r!zV%ZdmTT=G{zrTror#pGiQA$K&qc?>4Y9(&$| zu;=afzZu_u@eJVv=9yifoZJUDy!ax8zQ*@1G{7xxM2@Fj=+;+x_cz3xGoPf96x?Vg zj08bk-783P4RJOv3kn4Rx|UWTFdzLmHv@(xUKC0yH)@|UWt)uJ3TKcq0DZShksQ&l z1C)1>2inh*RAdr_%kc1%)){Ox(FVcKqu+$sE`G|z*P1maZ;@-(@O!>!BClx8(jb3N z;B;eDg&mT^i|i}YywDf;EWtxLNr+?`h;5pHk=fT2#d5E~f!q|oO!P%%_ZoH3Ir#c- zIi-Kvxw_y9PiUtcjzRw=7RCatIPaCr`tv6tYH4|@*ay{!68INKhAYovlW-<8xFcZ_ zOhI38+<4{mLdu7-Ibh(QN%|rqVvZ6#3J(tyB{E&XhrcFwbLr%O<}$GI`0g+Ut-0WM zGV{k8_$P(Qr6?YLTmfN9=3Q=V5|my5A=JB{SYR95b@cA1kU?xL{IznM7GgL95M$=* zEAbNNF`3jLs8QogOM`wx$Z7`nF+-dX37@!=4d*hD6<(eG2T>ljDd);Lc$qiuxh{aR z!{mjmLnj!F{@=u&(@#B9#pg=i{(p!)Lt*)vAy1$ii{QhtO^F~uBJORQw)#&sX(%YX z9p{Ec8Hd!~Qc*nE7RSg^2Xo28*8EgpIig=Zt;9 z_X!e)V0xSb?(lhS5~U;7z;IF=^DwZ5XjA3O?pX#r0y-+lSFBsE@L_Mv2;7*-`Xdx} zsE7igjt=0|ixmGk$~MFYE0*K~tAIIWS8yX=sa-DpEbN48Z{JyL?TpC+b|SXv>cCT+ zr?GQMr#w1%Ne*%(E+s|ro-l;!qhJx?P+;U!wLs}h2jsiFSt_>?nh%NKb5POG01=3^ zR%tHEpaO`(t=PN$u}k%cALSSfFHyV1f4VjrnN8@XMo;d*$O=s)I_1uYddqL(bpz+m z)4dVza1QW*)t~yU5BB*j69lmkK_+rRpyv=ah@^(AVl)@GK066P`C`DnVIg$U%~wVg zl3B}vhdfzwt@2qTsLY}v5j>S3)SC~3U=_MRp&YSkwFxATwQa!sXju0@HwDrKdsv47 zw-1ii*mZBw7d)+d4JDPsmARt!pn#?FeFPXs7OVdIzaj&2zIY7t={5?(I4O59^LY>{ zYq-0)<$y{|?q$(IdC=rHPS}O`fcyM|28cgqAra8!DXfSd0-$)!!T`cM|I4pU0nufj z=tbBP%F9uVnvdqrThg!GgihmG0IW2T!auf*0wXQ*bWOZ1++i9ZZFBdPYct3lXD5`NxX$Dedhy@LS*MF$zZ`I|a` zKT){QuxC4Da?kq$TPFCXmM}Uy(mW1ZqZo!q; zSxM>?b^~#1lsf5+rNIVa$Q0?GnKSYdRj|Ul(@VYl(+*P#?{1b{>woj#6u6lFVqGuP zs95|~gysYhqPx*bL`$RW3L!T)P5266m#|zlD&6y}tY!>e<9H$<#YcZY3Lr@F>u7Ma z`#3AB?_|#zeKxk}&A>@vVnb9^P@LYGI1z9~XlMp>{`DHR`ik?6A)IfuPJ?-YiudP$ ziVwf6o1NnFD7r1hg4`+qs0i~AcGv=4^$MB_Tz_^Rl8U~Gh_I725G}vDeN88U^A=o) z7ELo+?9nB70(bVTSo`oPtnH{>lU$pJQl2~~`6x(^aE?cPl0buO5Ln*_h>Bj}T>@#L zgl>>DDO~)>o?y_{V;wz}By4q>#+x+~H9SOr60?jtqkzv!Q-}tH%vPZQH4kRlU25ZE zu9jC-m`J>|K=2YMuO1{`lgovuRR0AMq*>~Qjl25B_f^RG+l*|X86mH}n}Waq?8`{r zKR^FqahJiOKe?Lv8SJz-WH76>C%oTe7HuMrG`2f!JO7P95;c3kj~C5;d0mxy#|j!J zo^{z#E`>O(g8I&nB`c@}UPbw^iOxMY69~n&$_ha~tIwX_lOX0mi%hWMUe{3?A$bVv zU1lNPDAG55!$llm9z`a_OVkW+L#G!pk4+p&i2zO~{@wO18wX1*KTv+Uf~uzV+ueOb zEyP}#d`NY^1`wuIt|wHw*W{qV%*Q8q3N>jweKN#5?hUMqHKlY|OJmbn_5_bM-}6EZ zkuu}wLD$dyiYwp=x#6NiIhHbO_^GIY0!zzBjTQB)`IZM4@2bu6%0=!xPXZ9}4&%zH zASE!Qtb(QFV74Ld;)9K?P}6;K3=j|0N3d3SSCv)iqA)tAEBsvMgkKLy1~a>UX>Cme zd-`+JWm2j-xesQJ=3J!m=2N}a$eN(u<}h>{Anc1Q)jKG(zJlocHIOL)U8~wimA@;x zEja*cJ9}3)BKHfamyDd72vz*e>(iiGi78kNxCMCTY)Adh;xD?O9C*15l~M-W%N2r{ zLB;pD`<4GjS5wa@ax!R9K>g{&rTj}!W+~l(Om>a0gPp4yHgwhDLza=SqIw3@7H6Z* zJEQgi_6!H}%y9iqNtrz|Nih6n3_n<(bwS#AL$@z-4X8;ECi#u_ynSs&zY;>r{5erW z*-Rqbv7dz+7)Gvb(-3>?Qv%3x>a|kjm-v(}FsV1XA{&Z@|2DCN7G2d~hU$#?k*Do4 zxwkTa;1O3IWN1^+=pw>uRJ47m^oMD9v#%{T`x8p6LHe#B>CQ9^X4XO# zKLe27cZY%S*RA9lUxHH}ejba(5Ca=epulFE1EHn>{cLH4e2wa5(L>m7rPZ==G4PGT zFf^c+oNDb27`jd%%u1T=D42}@aYA|{U1%b5_egU>TPloq>0@38w#cQu#bjRW`4M3d=C34@7msL@aGPMC>e(`!yORZ`c$dgOzbuJ z@*8<$tK@l?`=_g~#u0EpH8I2oP6gIC{0-cbd8*_RR7>l3*A3}kC@ry8olS6n5{jK4 zbw20|nmUv;9hwCM`ihH$Tavjq;IS$k4)NZeoh+UhaTuSmDk+8XDpcT4Ju(xpznxb9 zrDmVspe!7tI4cffsr1rqBU0bqhxfYE3tNU^VQ@HEPLHO^g5sfHvkI%ch3dZYQ1^Y< ztlb_5R|d3~UZxH|i+yrdoC5r=&<%kdbucf%ES|5#CqgJl5lN#l6OFEyx4eudt5smT_x%E5|%shspHZ_QyQ z_L(f>)n_{sO(sr$im#**e-@0oX#Q*qNWWeQ;9pdAh7wzwsL*pPNlQP0nyBJ5_#+*` z&o~(pCg{e~IGZh5h)6xy`Y5l)Q#s>u1TGo|tdCxk#iy*Mvwe2=WS)9~k&}b|H1Okk zW4NJm{@Rj?<_qhM?m)bQ+K~)Ww$k~(FYx!Cg35Vny1v3MCD{r9HGZgCK|1!*CwYcU zdU;XlLi+gLPshe&B6(6UhIkKkaK`WfGZe@UcKy1HahjZY{`kAXf(5Rk0 z!8K{qPc8Js`AJfD0CIAWL+CzXJ_>t~346fFr-nKWfrAA2IJZf+1VBWY&mcn4`qgX` zRSst1+^~o?FFRg!tqO;6DxiVUvR1%^&~vdWK+zx+y0uZjbDzOSDe|LouFBjYyangh zKN>MXApoS42X~cR!$HL~PfC79Ls}YJ@y*5~Jg{>@*nZ%QlwkS(!Rm^iFdGB&EOOs3 zmGys4J}l-pzt2PWWePapS)d6z)?0m+ZH$JXC7y}<1t~swMpV6BjHpDY3R#>`oA42Q z8$hkq4O4cpKy>USl&(wtTgxu+GFAwUCWpP^)^A4v##i|32&a~K>}mj*PP!EGV{%0p z3mQvrrx|QRB)oKE2?fi}@&FvM(#wfpw}6BbLfcpTI>4QAv1x?v#%up?G8hzBpia*J z64!$@_`g8U|NqKhAiD_YBjG8y@_+xOA@t)a$1Z+DMTjs$2(}*IIyNAM^G)w4h$QxZ z04-&9W`@fo!$y-0sdb_HGn%cfBw@u4EUoNDpl>(jFB^bSoA1B7r4LHYLlpl2edmd!x6AFj8!uCl z&w)sG`u2U4UZd18Q1(0mDf$<2z9FF9KkKg+021kd6GMzQys~s=U!A;;vOx|0=hD8y zyTv1a()R;FFb!deGGE5w<$Iq-6iVj;5opwso*SC$Nlst4qdbw}IkFQxaO&OL8G5np z1Ow~^gU5=3HywP%S*`bPsZ{x^rYJtqE9Nt<-_@jOiY4D0TP8b*vd8dru!b7qzueN} z(2GCs^|ze_Zec9TIQM83RweCf?cyQ63ru?WE-mcUSsp2m6)(_NpM~jJ)x%S(#jpJ? z{$bL|GNr+DH|MHK48VH9JTx9#SjZFk5?Gdq2tMFZSFkON-rmpo>FMbadOPuQP_s$M z$O#hGng9^NJz0ISrvV7##k0_EH|cNIeXt-pup8=qWU51?K ztO1lkioU0~s_*t%(~o*y4cv1tH_f5KooAv+RS26+cjGJ_pv@Oi(J(*`Pqd^R0CF;= znk@fdwCZhhbB!!5O=!^l#Cm>Gi?^CBl)VApg*!FubMz<4v4H)$y{Jw1Nf(J9V-VYM z(bT>D+w1^<0nlHv>s3?7Q1J*Ws1`r{;n@O$W6}rbdk-J|hqS@@rQ3yZz%!53(G|m} z1o4@u>ydx21@H}Y;TJw<5EWyc$E6L?kB10I*hqoc(;Bh2yG@-_Ic47trTZ+zUHBVJPe5lym+20QNfnnUBcGgPMosuPGCVVn6G+;u$uDm| z6nfNRdL${Mg+QwF)3=0)KZRPN*vs$*_EHRWb-<~*s(${`F-B=gpJUf3VDC0Gne101 z3Rn`XZn>DI6o;z4+<_TMFGrZFeN`3KTRnpoFLwCa)J_u_w4D3-#RpcW1azZ1KrQ zn;S4d2n8S7XCwq9h!2PE+M~#dcJ6 z5;3l{KCLbwzxr1&Dw7G=t@tkLo;Re%Q9vyIBQwUOU9OT$mPr|}S~Y$rq^pu*WFKm$ zSuo&@);GDIkos{)CBUjB*l4^OD`7`!vmQ}~sbuJ}f773}#H(Iky{s_kWtBMy= z=L0_lwfi+nF6$o!0km}6xc!{9m9~(m5q~ z>pO;+f_a3vccNUEpmT;{v#9;I_M0@`gN(VaM+!TjnsXzPX7l1wuS$2>UnL5pxB097 zFv{AZ(>$ngY9~Vu{|vJR21?;2XSfAdvJ4PES`sJ5BlezD#An=W`x;&3Gaubiy>kO8 zj2g}AXoUU7hCZaSD5&~V9mO7YJRW@I`{G*4jscEY`CM%G0>fqmGj6NR#C*=|(#*G0 z*h*BS*mKq^&IvBSrC@)BSI}aE@&HIhT!F}w44<7;NeDwGt?)B;Ts8I^N4s&&PKK}1 ztBQPvY@-E>H>uY*t~&XR{*|!!O) z*wrRsU-0Lq@0&2San{K1if@>_Kd_tLaQVvs^N00b?$xvD$M`e~`))Y9w{vG_emTAl zFDyt9P2$0ZiMp#=a?_0@`JBLY_@a-B-&dl38<|e_6kR`H{)Ul;s`;;x-b7q@MVU%1 zpke$`>?*uo0$WkRab02{icB1|9n3S;?Y=Y)5t2XFl!Cq%Mq=Mi*966#ME=Kt*_s<~ zQ)Cn#{%PVFfnU3o!D#>a4We|LmFsaG!D}J3KO_;>bE~#IH&qzKnOnlvqZIo&+#Lxx z5FDXV<^mH=-I%B~+oaZZt~6xhBM;ojCqX{m6L1BUy@|#fJTQ z@%9$Roz@jezVAZCKFn8AhvbZL2Cg*}&X0-DX&)(^wU&3nw*#X6DSVh3sRJ0Z{+uz{ zZ1W4Viv%~1D7BjT@3024QMUK8Jd2ds_;p`SZ7U@^HM0nNuK6L%c!i;L>;pYjS18-$ zK=XZHp{^D!fy!x8u@Pnr_YdlO(5TLo;GQ(D$Z&asHPYJ7nWJHLhCN;6gs}&t>zRwpM>GAg2x;a(_<*qPrcpq1BX(>)8 zD#m?RBD-v(aL045Zk#LJRQPxt(7LOu;>ZmsxyMK5`g=sQ*z z(2aze&n=1dd;=r_#7Px)2;MRh-(5br*P6p9T?u4JNgO!&NiyMK`X6EO z)(cE?i++2byPqo#jb&pASiAM<1OzE$)*gm#{2eYY@^gsrvnbC@M9Lfx>Rvgyml!3( zvGnV{%yJ6OCxVC~D78j+6&Zm?1l&!L4_J4DZ6#I&D!d=(mJ2|tBKF#SkctZIz$afy zEKvk$X!d{FdvCsBbZtvW`l-lJJ;$RVrHnnrVh|pW%q4i({%OZ~s!+-YSCW&|!_F`VOfLQRo0yRhPtm3L~eio52J`Nce~`*|BOQ##L) zTeHx5?pgN*b0X*h2`D2QrRucuXKO+whD0 z83Z64n+^3@qwiEtBSp|1%!XGP%Uy8J6M|6AEaJSvQ)s7D7m03%A**=D*DZe741xF- zKS?6gyO+#NIx+|@^7DTu7ogZ^wZkoF6(Rh>{NBT>K=t}#ZEsx}I0S-+&LsldVu7G` z7?_W5IT8XL2ZPWIg&t{V>PN8>DxodKSDO{Ht#d4|tWC|6uIg5S$r) z;w7;L)w&5?V=w?fu&6+^^$}b4O1X44e7ZT+o=;~{fSdz_md(WR7Rh!=85;qi=Pe=7 zbc!`<*Y)5P=6!0eJo$Sz1*sqB9>j-^xa(oz(%Phke%7cIMU9hX#&-`p;fLmnP zI_?Xs(H9KkNVC8`46RdrO8S7iu}PYPE*T|@?BqGxr=P-^3Wzo_FZ+&?ThzayF_f~p z$AwDHTzPPwsEe(h9`l;)5Cu_Nqzy;l!UR8Y_^CXiFoNP+Cs7&{CAl}1T{0uXW~0&} zR#|ABnxPH`>S16x{!Z7TeSdRNB76bN5g6dRc@a9Q3}F$6LnyI9|58R^*s37JG!;@& zu7Jw9e-ws>nF(z9l~fbHfb~M0vj{*CsF3vmbe71qddvyRs9_2G7-1YCY-B6Gkj)AJ z67EZ^kuN!J#Xn49pNxuviUqn`6YvU*3o$Jk$WtCf8qQOtS965%_2D>n*Bl-|a}loa z<5h=sKxwBv`%7asuo=Z?ZU}vM@r$%Z7KsDSQla&cqxNb_Em?(CrH`yKg`-)j{DV(mkTeymH_2vSI#{lcFJK?4h-b#XYRtrOzL#T^bB9B}Yi{R2Lc0p}YaJ`uWow8f#m%RL-We*3kVu3qSB@=ju z<@a-N<6s;7@RGv3p;{?EL(CJdDTyQVTTqQV?R{OV#kw1>v110>H{rAY7T71R$bSD$ zBjaCNKkt(<+c6RWDY(t5Ah!1e!pNpJWM)D95ia-YkZ(!N|^7;_vL);<(t2ym%2w*hCv}2prj2Q~Z$!vi*m*&sO4^ zodfys`~wD~GwHCXMN8+`wX*-kr(X*kGd;f`nFl95_lR?++EdA2^y0KjzN?=U5nQc8 z7f25aZw0IMSL3|V4}QCPOc6thHIRuv)J+HS9%646gW)msNNZ!*ke`Q!F90JDySd}e{5aQQVo>Tr~VM*!~|ncZcyK$7dH++l%$IbdnY zo;KqD@5NzPRBR)n)PqOd+K!J$Cgjq|@!xt;w-aEXN1y|&&dW>X0^L{-=x3^Rkf4Cr zxui-#8G2vgZ@Ed9ws|hudw^a?H8s4=Hvb&v02pWh_~iM3=lJw=CoMSAT3F1<@WF!! zy$M7gJSdhW4~toJ04-)A&W?Q51VgmU3Wi9=@6u&#>}UxG!7iQ0+@-HUW5m zx7j+pJf!@luwS5l51`Z|o<_lP>}3G8pHa^{C*!Wcn*0^X1cuOWy$FfOJc#pV;YF@; zYWUuV#0t@c4Xy_fk{`Z)_`f<5QfKtr3%|)kfV$%e%U;t<{n8c{*rkJ8bPsZaz`X(3 zyRjM?L$r`Y<^u5IbK8NZ;C%SQpMTFT?kUOz(h0OL>VoRojiLGL0Vv>cE}6E&zWifF zSjef;ez1E)=Io7v!aSRk>$ib&l!FSY^Ypl{{_N4g1vGOK@Y6T4d(F|EJ^;u6v@B;2 z0YX9p%`9y(x$~{q4K_$9WhID4X1G@HHaa_`CGc7T+b?2aJ>OBG^{jUXLJ_oF^r5y3 z0I5S{kAu;fPLIi9LAxu<2`}Z&GI0z&=oxJ9y|dN3s8t+Bb~s+jJ%>+7Iu5I@o=J@U zn3!;5SLcm-Hn8$Z4dG4fYu-3n1Ak&B{SF3H+O)mSoq7-$3Vzjij%pUXFD$Vdtm>cM zt3>!{*Z51CssF04^Qb6(I!@4o4#W%5Z7c>>!VVu`7#$140W3(-2blb2%E*PP@In~X z{8XbC4qFi25je^?5*U*^9Q)iD&iadRly7}|p9}af`|-z^<8Ss#*(x=#MGa^nJ_FwK zJ5T8$P`%fJD7+YOz5IxX2&0#wDyX=xOauntg2=ECE}b*)=-C^cV(c)a-{$Un7ds`D zt6kf_eD60GqgSy;O-1PMysvfhQa;76z4r5eZ^a&*bzoo z9=%C}{*-c1`H)`m&P2jlr^SNhC;b|qKh7WjCA`)jae0<=Ez-n9bS);LH)7PRZZXuR z*f&jqZhSHF{Vqju?fdyigH+7e`tRGi-r+x68XDfWB(v;qzqaXCrNWEc^aOR?cNF91 zSzdlB2+TeY%g{FXz0Mr291Qc!0q*YCk(ySSctP@|5MEB>Q_Ig5Gsw2LO+qBYL5KLp z|3-(Hp&?Xk(VGS@(v9v2J|Pu6Bb{Eg9G*c2&v;m3X^`_b94akvn_^UTC+Lnm$ko|M z=1s>SIC6gKKrx~F%YIff-$g|}4N3K=##XV%8Je3yE}ZSrI^O!?LkkHxQ_2n1qXL>g zImZff<|DRtJ?bRCAlYciz06PD1d^H&vjul#{Rz+Mw>gbzy29ZHZk^@UcjZN_a5TjX z_h!pi#wN36N`1cB{oX~!!SOMZ_N4}p54N{zUxn?^Fj3?a&R@>l)6_VW711@6e1Dm( zJ9J2Zyryl-(gw2!D$G7fixHAe+A^EGw)DmV8bdc{|M@(6&O!aP4t5_Ie z&{q=YGOdyNEDNj&GJE0ITc8R`&zn$b#X!X&p;BXRw?ig=h)i;nJ1W(+{h^xIe#^R8pj4H}#|X+WK#jXkp>8^*&*{B-+;ewA#@aXO9gPD?XUPx2@`T zY}ti`ZIM(6T$9<9bn*6Sepp<0Y>2T^apmJPuYejyG~B?X5Zjj)NJ&3( zye&R!x2{9xZX%SP#ExkLi9MpQPjMos;44771EM_j$H>G|2N6 zD>?AL@r)vq>pCAFglyYjmFJUBX34`@MePFob!WfIiyS}Qj($FMeCp`; zedQCnQnPd{E-u`x%%}MFmcMmIeH)X06?txouwy9!zR0HJ7TRV*6Giv4@ZX5y$85cu z=I?->wZ=UoOmgoFlTA(ffPbF#?QCb<`KLVNaZApcU6$!{|Aq|lL)(LAIUSK+x}$3t ztVW-Jqj!_XR!8Jk9d|p3NvDoi1@)tW^)Ycb*EkCye>8{4hT~`Qr5`A=!Og#tjq`)| z6%OY7bN*!TVwQJ`GvYailZ1AM4|jlsdPirTWNyF`oAIo%Q2XsvVGH zf&#PJ_6z+4VCFH86R&3On9-cME<0{)f=k9xGl8n#dbsQ|eo6suu2DKLYlTC56JD@` z7eqbkNR2kvYJuTAYVy~-0nR-Lc&_AbLYH|#7lh}JE(jlqbEiG@6XyTXnhr9yraJ$S zXlep72w5fypbI9IlwT##0wcsOV(S4WAN)n4exSNq4lx8d00L=1ncad|M{M)v`G-oQ ziBiaU7D5Y{qfXcwS!)73YI9b=7Rs@M@TktuG7sy}qp;JNRPp&#l2YX5HE+_Rry0Z3 ze)yD&5%ipdr#ak0D5UpEIH1#``KF%Oyr3~5=D`6X0ThcI2Cj|7M`CTfAOf%;%A^U; zfh_C{i*Ters~w6o(C4oOl6Rht?_$Isdi~l>uvOEg+X#KqUyJpl*1V4~rPaR}VAeVE zPv-2G9o8Xe@_SqrWJID*v^}AtO&t z4lgJ-H|a5cc5YTLe@d}@mf|Z|P_BI0*C6dt^T`qB+%&eSnD{(`xfM`-pc6KpA$;wQ z<#H2jWj&2~06NY|PF9i3uetR(?W>(5S7P;L?4_hpvenM^@k42~ntBCQP%Rk7=>F7< zQ3TJyd;C#?>4q^0+92H;+Q zvzAEk2#nx_zw1kVW&&Z;b2#DDx3lWRQLF$nu8Q>so#;X3+T>O5j7z=(-iZnIbw3RV zm>Z3eK(ePHs)2&WK${BmWus8V4E$8ML;K#twLmD5uWRZ7QPk1+Tgo4e>U4tsxZgyc z@<9kxvj*wtfst2dr2$dsCpf2nt57lL1b{ufje@;XqYw4O{smEfT^kz9>-(!zZA<;1 zl&aSa6|pgrG$9fNE|0^U3R*^PW3)bd3gVy|Se?&ZMq^mOnquaMt_Djm1<$@&vsnP&q$TEP|k-|9R)!_HxwArBS!TY6CM_2z+ zTM!qpr1u1XxxUtPm@%iw_;ns?;pjSDE})7JGNgR-@i$>r*ukJ{1zU|3Iy*s^%Rfncz5dyWa)CgRK;%f6QHkHgU1|(sKoz#b(56fe zkbju{>7!AUmWA>0xd=c3 zpf92ZS`{&IA?0)PWj<4rmARU-pU&!cf zq4U45J9*bCzwjkuuw5?ioAn>H-gnnqud4fFbdE(6kgk+mjXQo!|S7O3&ySR{1rcu=2AOq)X;&?$H`O$Gyy(KLXqD+4~53 z!zz7fYS2ad-P|Fa-=Y)hb`_&(IoiRxBumd2ZAldMv6{WTt)TB?Qm zv`_2@=mB&$)Vp?ruo75`r-|hOm6B{1LMkZx-J{-(2AZ`FzPy3K*JUZMJUVuX>`vYF zX8c2%8P5ZsGX=AszGfC4%$7cII~@N}o#(sYxAAr#+iETYQoYk+(-BE6o})=pw_grjzjLhfcf}qBXUQvkjgEM}J|M9&NQ5-qIh+rp zC!1eTt3I%6xJHwyInI5FYxhT7LDvt)W4D#?*+@AWyHpa18e$F*KbDomD@yuhU-smy zs^TyHr44Zj-QtJ2<`kOYjaO}rZUkl1<)>@8e-@XrZ@RM{A-Sw1V$+k^`hwX9hENqU z@fG81pc&j2J%FYaa;PO<=S$w(oMt2(JA=qWH$GK4nXZAn^+{*t;e>NR@2I>GeL;F- z{LT|MsiIx2aP)374XZ#WRt~2X`yd}iFQE(cd&hnDliv9+jiOE;i9W5Le1v5A2)rr> zk#l+ULz|nN%7Fx07Xs}pRXE<5iTVEEqV&2Dqt(X2!ux)sz_WOgeIYQ0Lm?_Zq)6p(6ToJQ0>@d6ffbzd0KhiPD{YjcC; z>BoN`2}wfAIukQgTNR*Xc$(-uFwkEnU+#J$(s6B(iG1zGQBAzyR)Bh}civv7D+z46 z#ecr+5a-8p3qZGuq)}{w)>zA?f0$o1o*++k7^=_&2>uG(>8c;e6~Ez(=YW>;U}p;? zA6%cl6Wj4Pj{=HL^1jaPht@YSq9o_Aqw$Kj@KrAorXp#b%W!SayrX*?jzi1mQF!LfrCV$S2T zBAyFIE5kNJ6}jTOtG!vYBv#hHXqz-IIX9QSgG`Wn$1hVV1;9MraXC{^rn&U?`L%|o zk;)-_iP#ky*hRmW9W_dU1kucJ`h-#jNnl5%iJZGSTO?~Tme`ka8eKoq(sqk*7=A%b zNFH|<`BUiUge=9jtKGLI%R1g$$kRJMwPu}{b22};`|WI6uewB6z1`tj9yhv1}|rEAKxfJm#SM~V&Z3@Mj&W%z9mc-2Dp zIb##F6*%_5-c!$Xc{qa|9NgJ@^>{uK6PdkEL6Lr9;*8U1ZR-vYX>|`AcRU9@4E7RR z1J9GSm;8a4e#qd!nKU4*^kr82VNOcR4Zn17FrGOWd{QSETI>eizuxUvY{KWem*~4; z1riAyj9&Nl6)cWdxY{Q(3K*o9xkD?)U5?9!ri16^%?Fz&82K0mz4)#TZfFT`RRKR} zxNPB{3aZ51{zBFwzsaUo6^&IFjDgBQTw!hgwbIqDN_^$`=VjVk!~HB8>36GwcTn}M z&@?nQ=;TJ{@AY}mj)kcLdVyni45>#e({23qM9ox8E6$Z)7^}QSF>IK5w8h8+f%-s0LxjLELYp1 zaE^B=#do<)l#$>W8LI-p(5biYZFmx{Hw6>E@Nt+h`&K!nA{Rp%Y`${d7Q+?u{Sl|q zTMj>POC+n|x_AT!+5(`jmlBi37mYv9+VbLJ?5EE{>#tvy5m(&3&zDhn*M10ETpYi3 zcoYBD+zQDi5*vE+?sKF+ojXQTE`i_Jp01~|GRg*oEpkOo49^VmTj#UT78hQ@pXX_P z@#CAB==@X9QNdquc|`C}?0w59jIpQP8ZbFvi8>9!d*d|;FBlqa&!lu18hR0O{aijE zL1g6k@w2Le7g_yK4<~H#9S6qeaP#|1DGx*pR1N*Xebk#GnLN^CT>)X5<_-v1P@f>xdr@B7Fbt$_HhVJEbaXg5T zAX&1q_}xP9FL916OtY_8G%oTvj@0-PDTo|?LF-JI3ZE|hYObuK)AD3+3lhrsNa)~W z*>hb4-;SX)#hH-7uY%*Fm{5{0sVDTL3K$;dwKLD~1>WGGSLw&VxR8ARS)NvhJs<#e zO?tkguzzFBbIcFJ_OZpqp79csoZPDBP(5@UUEU{)6#9p!=Qc`pK^Wp{RW%t#;2;X!3rRK>qiH_T4$CeKH<0P zn;uC#B*Z0PQ`ETf_Jj;UVnC)G=2Odv8jWAL5A-Ea4+xd8#f+0k4hecVgF4m|>9(4Q zh&C$u)nDfg8Jn9c?z@B}yDV?eWuIBs=lscnN3L`NZzsIPZzCY1S6E|?7z~p*QFCF! zgXt$p5$E$NsO#M*7x7O-)S90kk9;0YD*d95RFlKL?%TIkJ$AEP(PEw@BcSXwJ--xM zKy~M~_BXI!WnUlC1(Dxqrt&J`n(!i(K0?jUI}^ViV%q6Dw>bC2nLR)OQJfo~+dB32B`NFw=cOhF`>sDJj*V&Ao2A@ArhF>$a}B}8(h zM~>Q$zDwwWrgUr3PEr-;+F7V)R?*t)jo)h7g^n*mq#{(B6BxkQ&oZ5U=Xja^m+*%A z;Nj-iy7WVMMO{B1aYA=sb$9|bo$%WijEoiG=}uK;w+5Osj{iPLCf{BQC-+4d(dh^G zXLOXg`V_19H|B#L9~7JijhkZ?S8ohc|4yX&CtaZA35@NKr|cw@=88Of?Pp^dNx)$t z20z*-4j-oG9?2Co-eGk*$3nG6)^q|t4s+rY42n{^$Uwx$a^PKM0|-_QZ&kdVq{^dU z?pSO3wxv`v8Ldk4onpsg@A?#L6O~uaE>r`b{qRteyXC%yO3GtaGq z!;ULmLq;8YC644wcnz?GzdB+nqYe;HCLieU-05<%C1mKouXwUlul(Tya;wv&^)}X< zKCT{wsW5h#4ZO#Ay2dzVX?F;(-4STg6AqxT6J=t_fnO+63N_58OzK@k}}VkDuhDD zLJ=Y}nTkR(E0viP-gR!*eZ9Z;c%J9?{*L!Oj{BeM&iA{|?|H7_vp#E`YoQiM%S`I6 z&Xfm(D8B`=FqqbB`2BRrB_g~wm(st`?c&qs>cC=*%A_>QCY$=!_@~4BnM}7-Zu1iM zHP2nWpOaygV5Bv6`m^)OfdDn9^o9C8_r%t39U15rf0w|q)bhns@Q>yHZYQ1#Gc~;; z)BT`t^6D<`&p3hX?D3d=i3$Vp+DEULjLAT;pw9YOGgfN<<+%OYRHoxCp@$C!P_GOO zOLozDI;*sMu-=alDF_(TeMbjv+Nla$vJaRJhmGyY6ma6@%BWPo?EOolBa() zsbp|AG8O0FbUZ6&vKE2W7(0a-F8JFVbc0h7TzNRijX?_p@`d=IWLHC!}df4$ZWq2-U519s3^)5A7R5#(o&8AfO z708@Y>jn(E6X^K7P+_5NPht15j|Aawd|Vk8aKHG)fyJ_*TpSIc$+XWogST)71~wZD zUrT<&{ZipnU8fel0rg`ZvDMF^xj<0$h3`jA#!hpWgroaBt0H^esNoC_(o4I_k-to- zVmo^&-*BwnLUZTNfuqx~QBt2B#Nzf@WrlzoVAt}~KUkQBWe&44*n76Dot)kGkwrgs zVV>;6clRb-;@_@*H>3RdCC6D?Ynb$TXhEmhWT!%$;uQ`+o>x*`B9VP!!&cfzhFp|w z-|W?YyGUFuD!P8>2+x8Aa0D-o z6|fF3z%8nw^5VK+Uvz8oOwyS=A8q=j%5lwp76a%p?7-wU)1^v%#f;-ho&FOp?Zjly zbry@(gY8`8l|Qu74o}z!4QC%Zh&P#(rgEZ>8(R};)5SY=#V&THh^dFC%s^o2Xl^>@ za(i$5ohVLnXHG#A-*!`%__lvU=v&Cd^%YXNw`cQQLUi8EY}<@~y3@k-yNO+M^T3_f zOZVJA?xXVEugPF?Dc&{t&BDe#a=BvSNUY9;xyn-cV=7lM=htm2oomGd?mXPXGqS;X z{AZx1^2cd$H|s9|JGR%Qm|tG5-)sXRg8wbB5Gq$`7A?efCgO9XEM+OkGTs;b8B{Uk zSbC^qtejRCyf_an+HTTDbe3CujwqKV*uw3t7nG4EmpRA-O z{O-wcF7=mBt6caUH^Rx~##;`jjutPEKu~)0SMdqXRGW{Pt*DX)LxDTsS|iz}av}I< zIFHq3j@5SB<~9oviUalb>XAa43hQ@2XFYB_QqDs!;^e=6!S`@$;IM+)z}+hv5IvSy z1)Wba{mSYD{pJjE{BF*^uTM;cDQywW_pZxxbrp|L?rzVwCZ*}M5x-`Ow|pRGKRCMK zA{)M);cTS-Oa6t29R+4(K`$IXzgr_c_=@d>7BAOAUCIm7{g323Zf|Hc-1@7=?7wk& zcNZ*R7ZQoPzz(~h^l*q0c0mXQdrji>lnqId<0C=250e=F8az9L4LP$dpLsR#qi~Rb7Nt}v|4mGAiH^ffah=cp(c!R&Zk7iR2GA=Y9Qh0dGG&E{GfyPts z3xGn5bb-^yqVz&4w~(sPf)s_4uh5ne8Ya&bG)<<5!4m1hf)J~-EZ_x4fd{y?Gbm5~J<^}e8zv}k5J@unB zGK_k0-@v?4On{&7NSv9a$nz6If|pWcqFUe0isGCdh4#>tMJtAqkKK?Hx>j#7H__$N zdY=b{k!tD?%xpgy1_8J*CWD%2L1%zpY1deJj>`#QB){Myz6 z2YmntDFEkaWy9Sz*jPP8u9f;b>cF;TUN zf_PVBLpi5u$196J%fYUUw zVn^9XXRv zqG{O!0p`+)d&l;GUK)TWNY=RT~eeHfhQy} zH4KmU=OU6I__0rEL7JBiYdj1^(G8o0@fmpHBs}3Jl{4p2`WJ2k<~=!+YLh4}MV*o^ zdovBbp&5Q=#?w_soaQ||^tBs)IuP5&Qzud8Yi!m!2ehkGeH>k&%hn_}dnDxqacTZ& zX;Jd$xe1(z^Tb-eI1N8^7==-yaof#bFN`t39q$LS9#3~NxD7d^sL2U7Vnn=03V{$^ zos}_73o9J4sI=X@D4fU$nBIb;n-3)BcUfN7gN9?HbxgGC`HZ2`WHevMa*&5x(XSxxH_R|#RIT?2FjHM z58#Pg=)|0~`o3TN4R^_m&O8ur{r@2vcgW>{YP4^SFgT?O_}>T&C|-cc^t1;(jzxA(AQBN|W-u1Yn% z4v;1o|BHIYi9audY75M4~+-YX5H z>nhMXPvIij#nTV1SPA7w3KLT0WN8XV19tqb9O#Gcf*eQcm1ZGKI$P^%RMD&C$j5lHk}U`zO$RsJsL$F z{|ccO)$|5x5qkZ5Qj(0@bTWr|v4)aa2#f#5X+Ov{Jazp1;wfP5Y12;)xAzZ>H{BaR z6M){4zERfy3i@ccJkxE0Ze!XHmF66el2zDRIkyVeOCR_*CPX}qa*0TSZ+~}k{g~GT zD`1t0WZTGCtVSIbm^gpwC;9Z2vf;P(XNaGE`b2nG#ZbA>CGogAaR60BT?r5h&mf5~ z5{Q};+$SO4T!j|zbalbeV$hXxApaZ>%$L2ED(U~%({1HCG;}~iXr&&(-7c|9^B7*37ufx=rC^AFFr#BR4^NM;N90LX!@-WqGp7$ zE7b0IDd6x|Q6qn7L)0c6fe7jQEg|9|sK4Zaqmnv~YB?kOLcyEu&FRo*&?|Fbjoeri z1nW^rG3v~6<~2@kqxaRV8x_z%Yw&L+*LCQRr{*gI>8l83YfS;}1;y7g&}z9OMxb>E zwmcs?|FrqaL&-;HjG(U<{J;QQ7=JG7KKl@Qg62FBw_{iB2NQMQnribjq1o!IHaU24 zZ4xU{@~dgcb!95v%mf>%4|blmDP7~rE@3t|&drVuvAjYaR#ygn?&+lU=mVrqZLP$W zMAj$VS~$t^hz-*XjOoJ$6V+~*8zB#IIXCkhj(tH97ep6h`ay%m`Retx`Jw5-Bau;> zyxa@!FouB1Z>1-8KtX>emP@-I)tjRx^t4ppnKlONmAGo}IFN3!&+{%+8g5*?4;o_S zo~(u$;sFg=n~CNCo1qa2@-b=3GhhIm)L+Zr0+!D^&Zroi`a1uTgI#r4!}&h>lbZ|r z6QLjg3+;P$ zXOGj}_(8t!vO0lBC-ln%_`^t!x1KaD`w&zTi8@9GepD1JBKKRFdxe0m#`a*FPn9x) zLVWAs`f`K%IJA;006Tm1GwHFOYN&kgsF?m)!qtj?;-yCf5aX1GKPVUNFkNPaeZ^5L zXj6}UvIxibX|jF>V9>huw`3ihR4X{C6oFry=F9)VE!r3>!=uAcpL5yb5GnO>V5^oC zK*sFhD*5za|MA>3gRwpP#`me<8{d+b9M!rImvNiJ{7Cm;$(0D&8R(EYVo~@rlKNLe zBt|!x@=Y`R>M`}uMb2m{P#13&Wt& z_VqyxuXA3KkdQQLs+=|?vxlDi29UgpxeS9}l7bFIx%in!i&^#LrHM;T}quypl@7cI0c1&j?`N8)ij8gtzr-iW; za(nme91jK=7iAUMPkuESX<~-l&2-R27o>^m7nk??&GGK+BO%$@8UIE%{-53k^XdDB z4-Krop<-n^9)zP;X`)vZD6yI^d1)G8?{FkSLanLj$m45`G%#j1>m9St9tq?53oB8k zilo%|{6Cp)odvJ{{2X<>b@Nw;+()YUt`5N&1>4bO*EgfRe6*Y{$>r4$OUe|Klo)Nj z*;(JY3In$s9J0#kwK33$Ekh^y#Oa$~{BD_DiI6Z~x%_juO4M~y<2RJ|Uxk8`q)y4t z9g_^uw!s4Y%CryUM6W`Rf+T3`bQV>WKmt1gGP~-DrdH(0gq3<^xV&RnLVnR_Q$Pw$ zPjMab_^1i}ohG6CiDXoHE%Z-Q|MbFjkKWsAZoAYD~{ zD8TIOY?J2~r?(;fE&zmaR6%hAh3M6mXot~-c4+)8fSfVZ1Sph3y_<8ctk1hoXa!de{s;Op0`iK{x@rT2RjcGmzgre1`>w0RgQm{C zIogM>%s^`CIFeqNrS{h18BP==oB?WWh5=yS#d}V7XAhT~7d_iA#hP+Q3q)P;V zx6RanaRSijnuNnS<<4I+(*2}XWn>4VSffcWl*Tk;D%B*UIPv==B4wBJj(&*dXCfoy z)>%`sQv^CQFpwJAdEAWZ8tfSoEj<4+vY^n$xM0grM7^o7EB)lrb5H`q4R!owluWXQ zXo8W>^0kxP*sx9!mqRv657prP_E>choFZ!iHY}yp?bS<6H?z+?8w9;R4tK@E!V15i z7#71Iyob66+sGIf*`Zl6XrQ#sT%wS?_S52o&$hK6bpr=2PAS_526khQ6l}~hIRukn zeK{06RysLWKTl|>yC?+bZWl?s+`E^$b4vGnE9(N(d9;G3v|bw>0*LOeY48##)=rys&r7dJ{*At7KQ~a98ck^tPDe#mSK=SmX6c zLbP#;Bp7BqtEA5uRwqy~1hz>9m-H0Ar;(b~hPx|g9H222T(v=CS^D$ilS#ULWX`Lf(ERPV0_E0P9<3kKTmEm8nF_T)kEf4mI zVi`{d)@yO__U#49J_M4T?b+*hDs8{q@^t5Q7lBfrvaDw@r&Zh?@)pHZF=l-) z06XwNHRC3Hk~9sg&AXoKH&ZVVcOZ>H%I6u5-cB&Vn;!mk%en@ zuYOwh*xmhg1~)}_4-EeGfqLNJu-sZXB|!~;skj%{R!&Na_ZtKIP5ZEq6U(Rp8@K0j-ytk3-Us%3@Kv$uzFi@>M%)5XR}4r{ zEb86|T_xpHPTndifDA?{02Hm6u5^wYse4)RhcS)Qm7WwQ+4PRW_g1)M!Rw;WBE$51 zUSQfoy-w{>Qt}wD(FUn zMD&~2W;#6*(Q&&?au88g5Yf{;&t+kCUA7AwxAVQPA5k&vzE4v@I|!W>ryxkAACd5b za4Z4hkX9hqIM9eG7$~U@2$T!B;QSg=EhSbqI8SmH&Jnn30&mv}{3?lMB_XiGfvP@q zZpS+dEp%kaf2K5;XiGY+Vj+L{)GO2}b;vF^y~DKtMa|Fms55!Q4+rdQxkBk;+s)68 zN$dCbNx`U|$j-+Gt_fUf3rCQI#tT-PSf_`CF8!31%5zD zVxd;XH5C7NK@iwEdP-ZMB%}ahl+t0>cDh2J?Z3Zm$cLcSG%K_Tkob+7%=Mk^!3>8e zV&U7V6?3$3o+^R$;p$y=hj5B@aPB(4E*Mr2Kw+>Mn3aFbGy!IDd=O~vm461{F70C% zU@B{9D=F5{4jmpJtQRaOH9h2rk~09#Cvjb4eX$B0%vG4-)%EJRcHLRoInJJ}jfU*@ zy3m76e5hdY?fL0tkb)T06KzFI3SB98g+FhrBkXazvCM>tp=Om-4^0eb8zTcDx>!ld z0mqVz2eoX}H%guZTLY0u3-m-aR5khA*2dZ(9KbomxMjv5?_G{eXekUonDp>~f=)Cz zR%2&-YYMtqb@?RvJo=4hK>ddQx(b1x+M(Wv)aSLE_Fga*tJ5JBZtRMmp9JR|k4nj* zNfs+=XlZAn`^&&(214{`D34WzgqM7rC0Q1PqDF8xRZIpvy~_IjUIw}9w0R(75}N4G zcMR4-zjvGOGyAcR*%cik!v$8Apywf_@!y3##$W^&L4xn!$$VO3uzm%N(|C5=K>YbK zgPiL5yFF+2Qj546cbplRB9?^8LAk4%bL)>t3Q3rISGocNr?z1(;cfVSP}j-Nc$sRt}A|TbXVRU;b$c(;8f% zLZk)icpb1A4%1`Wyl3Z@L2VS)slTw{$-yRYn&uzEjDiIkkKjLn#o)t8Nv&)s)A~7S z)A!3EI{-yhQu!xWrP^5^`-k@HF_0d{bXU4eT!oPYv1ij|Q{+9}-|6OO|5}4W35Sp@ zsHkHVPNh)d+rHMzApZaX2Dou*I|+O+?MN<<)Lh9=KAn8l>~R`$j6N7m^)tzvZ0{|Q zS#Hn#0G8AfJk7gy%LLQ!|M7Ij4GeNH zSAuS2crX#J5Ia->bm?^Y7QFXBEb0&J9z2zkXwLUMIioEn*Uc_nTd3;ls94SWiBAcm8hai3Aw`QBdl|HA|82yS~DG z*HDUL&HG&)mWjJ3<$|=eT;zWU7x{2gop8^z&<0 zdx7Emo^17^@0rZ<-Wnf^E_Bvd1^ z^drLd>^-nAzT|kZiSMrr5`Fj=X7?6J!Q`vsuWH_Ovo`n~=6Y8LJzrDKL_Dwl##bie z>eTF>(ht_k)@A}a2T4>#6?|FMScdiv%?|I;B=MA*TVN=yZN~DVvs1aA6}yk$ zbEn#C86ul0XaPT?u~J&zf|meE=?}?g!E|!fUCOEYYsUnVau_7#^vfWb$A8!R#VWeAqfpygj9-Zr%&CT$BI<#*FswKMo;@7v}`erP;A^HMB ztk>2T`Br8{zI%%rS3W8g+Sg}&*}J?d{8XWNMccDy3re?KOwlLVGx8zE* z&1&w?n+z)b5z!QUoDbF0sF>vj1%i9e6(Ep(&S1SBjnkNdaUp!wr*xfGtJCZuqi>Tn z1@&-3e`X<%Dp?k=(z4|~#`eMa`frf(gZ=YF_V2Ck zfGNUFM5JeYfHKL8L8UTV^THGkjp{6_9H%gYxhU2E`^B{%p+w|wpvA5NCP@SdZ#2Kn z>;>JDU$uh)@AQ}V)=CVQ@Loy~Dl~`U$SEKz6QK)Rs`pC9bf(CaQ0hR#*T12dT4x(- zfK8F@hGvu3UUVXFo#DHrY39;I7m=4W9v7}}8L(AB9kdONxvwrS!TM=n{hCSDZ0~)M z{{Gq{0JvscIWU=l#nQ+=pBMuKLR5lbwNN`d1~yZIJt|z{0aWVS*nC1k30?X{3CB<3 zXrks-xL*IbEJ%FL;b{6@;G3Jv2ldR>2YMLfgljh*L-5vI{wUXT?tTWM2IHxAdd*Iu zuFOLw`~|OoT3&)_Jv{<~`58vR8zRR(#oXTXAguy@Z-sQrcFcQA8i%iokQ{LJ~Y z7NXtSM$ftjdknt)WQe8_ej%D9n2stvNE3kLOE43o)lY2R?jn}{VGdmi(XfR^>SVxn zimZ1)NcZivWQ$gW?V_1Atj3b!RMcR*7(NY}%v+50)X2-;`g!d9iT`t8O%#w10_rCK zVR(JI4dJ2Q1Co*|@uL~gg%I!|XZ^huQh?isKv%Ogq>7kKWGOJZgzh8%M_|{Jf8m&O z9qHp-Op}C2SlF|f;-eVo8wJK7wnqC4YaNZ?D0~+QW&p1^mLF;?xAYIv-B;Ud3`n;# zln3G92daQ{1(wS96W$@Dd!(xzl#??UkS>G&{l|y9$TFZcIncZp9v4@zd5DHx!`Uzr z(3jA`4;fXqP!a~IFBQ|AOuBpm%lcjgKqhxXRGyuQY`7btdEiq?SY<+C;%OB#m8g6e}W>%X*d~?TbHPpcMZ7n|1sF#nbQr3h3P!p|3KLz2Vw2W zaFF#~IoI(xur^;M(9i10zz$CpjK}r?0{7jkYeAA&8h2XdxF|7Ze=i4%nl8z;#)${P zqSW-B9Dqd~hec^FX8-Y*KI{pYiW}Y7_R&6S$Ppa2m+*uE?W_?5VktVsDg32T<1Ml+>+fsQQ&YI?vhLkX{O5;pSF z7n03PcM!bTpO-iH0lMY{bgkJJRlSS-x38iAu>Apf#{&uN49{_LaM(UbDNA8~QX5=U z(RcHE5d=Y599U5;sQb99v*?c7U5g0o!Jd7M@<7{nl1xjbl*p+_|07`)YCW)L4aU= zo6a~I${;kM?$iTv;{V`<88g_K~e}I zyG&T_db*XPJKD3;|F6_ZPub+z~FZ67RV9tw+p_s zxaVC(N-&22_xd9>23+GE#6fIOXaka(MjXVKeMN%KyJAwvnkbY zCm=qoDZl>ltne9>jhup3+;QMHcwbv9cm!V?G0`APgD6M##&ebt+u7{6v$2rOy4qfD z*QfTM=%8$U14BS@-*ql^8C+v9EZlR0oS%(^^c2|Yt%YE1{$0_$c$p}g)#Uwr3sCJX zv;;^qdFT|nAX0wg%SCWq3Mh8UhD$I@04}uPiVm9DAPUsz5L^_=hQ6FjFudaPU_rO| z2+%mLFrp@vL6xtepmy_{T6T{&=!u{V%r2RPVL&2YtIvV^YsM*UZLY?_5S_Oj4PJom zI=3K|(lXgj=Lr+ZREMDA{c2qxsp|(b+mswTlm?`}WtfK2jneW0+DXC{;2u6|MY`g{ zkZWIDpfv^=^9R?y`E_5CF|w;LNy&A<$fRoiy7G4`e2?C6{t@1((f-U6Q#M%3xLpDF8ZS0dr1ei`j1Z55P&SFs=x=*GsKJPYo z4to1|f}$^&wYyQziw}Y4Pyk(9_8a?z0ecIfOVLS~+Hgcg{DJ4v=Vz*8`h0xl=*r)J z;$mAeYg$h&Izwu^sRWZmL?1#GrG075Lc{*YWw%jKRt57K%nuE01-PGjs5jP=nFQ50 zgZ;~Od#EO-K5iQQ?a3DN*1+e1udHJBj=vqa14d=z7l%Ly3HC1=aCOaxYt4Y-K-6zG z#gxx@h_^&5_5ECND``Ilyoq4qMi161piM!r0h4|GVy}JIXCae$rSQ@tac|k zas66AzVOSmon~c5NI=?jIXAXOTQ)R*GB!wK1xP`N^ zLKn<)0w~VGs><%IL@|@yI4S=NRG{PhPa{*|k9b}nk}M+e-E8<9VBN=moPy4eSVnE& z3r$BOu5~6v2GyM#6WmKprdU~!swj`qb(*NYT*7?U9}I2kxne+%U*|<8B?-1bYoG8T zW->|vzS}JpZjE*2Ho55djm&b{qJ9Xo#vco&9&$g#PRxra(l54!3%Hq2Wn z*D(MmEEaEwB^4$I6y7U%gp3sDsRg@}f92bW_VWbP#v~D@={=4FmX$X+YU#xF z&bvC{r(0Gk{dZ=ms<{x_v(dn8wBz_Um!IUv*JZJH0byVbajbZ%#LNdVn5b=LonQpZ z8nIMYO@nmlhYm(Q3yK#oD6btZnI(?|&wxXT2gBc5{)oqyZo1ns>ev5PqKX7uz(i)& zPdp9oe0t}!N?8lf%Mo)dvwWn5ZUGwNJ_#l+31FZ#DH9BCS%O<@tuQ02Gqh#%QdgQx zDeyB41Xa}}Nyksge<1@e#UA-HIIB75PJ6YxS7KMc7sOTBkhQl7)g26^kTeIdx=1wy zBx=Lw>5bI|=~d@A>q5&Yw~v2Oj5H6m7ejyH&n4FTi-x^R8Ng2Rfq{P(sA`OB1h|=S zMD{=&82Bd;DoY-ySwC>{u%|mW;b$}T$3|B6qKxH_KVk7dBA!;F>5T%AokLFiLCvj; zP;rLli)7#WHTpQ$c+NJt;qL>Jq>K51eYJi*GSJ1aV{bbj&71@4mSHNlZiYaaZMS68 z4)72*-@QRxI=S2VFpecn^fnC1PKk1m>@O47`E>^qMeo0}wQdb&GKoRVrUm9#vZHaI zzVLs%J3F$2u7bTwe1BGfO%JjLZi4;1dk7l<1!-6;FCPcQ<_nN+`cGX2*}wd45y8lu zB{)n=Gv>Jgrh-DwyKMkE!Ejc0cLJ11a6GHZc;QJ-d!B0_+%&Y2)c7!U;Q^W&3`1tH`2jM=#+QQUNr)*$Vamz;6i% z&j0t@+Y-A(+Km%oyd(`dhp~n|>Ed>=0xt%wAm~(ob_Rwti23|H)Rleik$UD_0n6vR zCRGW#Y0^DuFLHQpok^E9LetQCtj*9Mk`$)+RFgN~=b5({&``C}($WTB3+0FA03q$~ z#;{-(5!-liyTKfgP!%kobBaav`zVE6^(jOyGOK1DHQ)&j3%&G&I*NSO;SrjdBnf6u zj^BfdCA}$2%*~yOl?KbG38-A-Q#l#rDgc#xPM1Sg08$ttVzssW)>Xh>{-(?@*xNj% z5FCb>-YoV77$-Dvthp9yAX=f#xQ)%<*T8AnXeM>B_gb4*cTDYg}}SCcw_bf(_beHFKhLo-NhNd z_7X!?1O^uZNlJ$H@La-R**Rgel`0Jn0PK&1&2bT;yFr!#^sJ!}@K1v@ws`9EA1_M- z`+W&ki#?fCbefg5?$Yg#@RmTuDR_TES-AGuUFYdSYT9;oFLh3fGYk6HWl-wW=8Qx( z_)ai8>+NGa(oyvIk>L|67Z4I zpsI~k{a?-`!W0kZD!A-prJZ(K#o@x8ukb-ZfaYlDEvokt0zpSE?m-bu0-#XiEfYtVv7VN zx=G_NPop5BhrsTC;$}`m#4GDn!wGk8JyWD^WSsj9Wa4q?a_3^slQ5vBNXAU(ZSmw{ zjkhKwO)eN~ULd|r55Yl?x-4E;^FMi9aEoCak}R-rJ=tGd>-R?s z#u}u3(6p|gPL_6g+>s_z_M(AHOAt(YFBGTK=vIMT!ly=X!-@7Z+EKxYiouDd)tC(5 z+?Bbe&M6`6V~!KxHdDYRmF+bg)F<4T+(&?DL>cJ!JqI0Ki^*l-ZT zLm-Av>RKgk_Egnn?Yyi3k0J3uSp?UwHU@|;w#sehgK_1~88z$xBW&l%fUER!d%0 z`yjQP2u=KprL<-A&UhEKm%CRP7v=+%^X|=lfnvo%BfWz}!C^=ClfBpwLP^e8P-!nG z;Dpe$Cw7>hrN0!`mt~62Gs5b*s3+0Cxba1SeoNxp^$RpBHcHw*2+W z^ir+|+Hrm42FZ+|tr<2JfKrNK_lGaSko$w&_VvWm{WlmT6d&HiRDUg{Aocd$glejA zYz6VS!``SvV@ zuB)O?I35KIy!>{BX7|Oq@ZueX1|@hg3%oeM_iGys;+CTLqx7PZuq#&u)2H+WpqIcI z=z(x?o_gI%=~6=o`D1q7lI-z=Jdf!(-pZp5*am86JX5zA2Syj1PoL$Ne)|-l4J>yy zO{yTdV*lgt6zf<#no6Q-yLd<&N79tn_&sCO*w2EGZ9L&0Gvbfll60yVECaY_k-i%{ z{Z{F_F3en3nE|Jws;Fzx8;4}^R zLW}KkV0jwvfoxpdw$H8rh6)vD(qC}zQClOtVAYGQ147ao8e(!~exEa|rWQV4i*;x( zr|`M`{+ve0b@W~SD5Y|Tr5FPmxeo+a?yq4)!vDSVbx`c>fUSkmzOg)l6^@^cp{qZ? zZ0emumY9Jsxlh^`GCl$#Te}qgyuIWYaBfFHm&o0CM?lcnxge=@=*Bs|TRB-kXf*kW zf6~BAt=Y}>WZi*Mw~83<9%C%_DV>+=nfEF34+h;&CQ*DLI)F&97}_2PN6{rX(rof=Oqn(*#mSF4LT^5kSJ z>2p~D8bo7;7lc*yEalvU7n&hYx|F;2Zddx3^!l|tf;Tx=q&oeaxt1nH zDU;mi2KFrF(z zpzq0h4_eIDkd9G1Z3J>ysrp5Y=h{QgJZzq*uo9tsuh8ANZ^%=Ph+e46;@f$dP4W1jD77 z^$}|OSoQef%L*s|3eFVPA!fS;QVcV~G6LEX1Q;;b1_`5fQB#;JdBh`5+<0D4zr*bt%rXQNazg|qdl{I za3cvaB`|}tbYuLU)Eiq<64Lzy%{cB>$fyg-_vo{ zrDZ=3x*DVyM;?OF+hBMQ$0?$rYIh! zrkIj0cU*qhU8mOt+ArK~NWj45MJX0-(606h+pP4@@M|eQWP}Lj=_A#P;0k^qSKzZ} zHwVT^B_SWAsEv^=?jMr-+8diFV?Nr$bzU~>4%nSa9(>M1mJ$_@bJaS7gc2(%ptE2k zdWK6-JXOA|xqJ@55ES*WZT(r-6+5??L4%PER##T4JEAQq0a1day);*~1#plH+ z7R|y%1Q@{lJ14oT?mU>(Z%>08KvnHvaav_x$&rUw0wk)$zWC;!+({wRy*Kw}Pn+Wm zUtL9MKM)If$9u3g@kqs*AS!m<313z1Iq)!CehRtG@GkUq6=lm6*+dIuLUqNY0M}o7 z^*Z9G%DkY?pXphxl>o#rin{Om3_ItURWh8jax9ZfXt<>C5YnX&df;LdN}j0Z^EtB%}xCb)CFXY9Judya10 ztOIlU0X-=35ExqOxSJfOg?V8n=kOw}%_QioIx1}oan5H1yK)Td%D4p|+TnEf*fgj? zAqun}55Yd_+{Y{ZFe%H(_SqrON_(!rx^sxredvv3*j9}kWEfR(wJz7}LdeNSPt@_w z>JL9n^smMjLWof*7We=}DV&^jftc^g14<~Nn%aOAa0ENZ*CCl#ZAG$&JyG2d8fN`e zJa`@tZCB*a`hiWO6CUb4lXMua#H^4!&V@OMB$z0k$h!OggMHmC0+^=0u0v4dL}sU{ zq$8%1pv9&aOM=nW6qgGObc2PbapV9PXRSvIcPm1|$>a5FuY5Q)!9LNU0unW7ZdBIj zWj^fq84?1|==R%9Pn1>_4!A4`2WbJzo?eQ2j2vQ|u7a7`{Ug+>nfbJsyPSHRv4WBp z&rb3y=E}idP2^uW!5;%B&DmS%jwpmMz{%{zX9_;-0ukDj_u^QqGnc>@OiQo2rb8y} z8RQYOEF?2PVm)XeRZPp;#HNR8zKrVF`C-qz;m7;mV7N#jn~82Lp}wQ7d%XaP#dGWB zw;+cCVY<&;xF=+e`(gRHP9cV9|G|D8p<_Vc76);hFageMx8`oiLM;jzz5f6#8|X2A zI|qv6te0aTAD#!D`E?zKdC4Gi2Jg7dXbs0%PPBo`L?UFe=I9D9DVC4g%e`+(mEKGR z6gxeTxsMFDp-w_*K9zY%-xYpcv*^Yg1L#!-#2oT{HXmofkHIvf76#k29tVYTD#9Rb zGF&~^fqQ|UsMaP*#=o**zp;b79af-Fe#nR9twGfWjtvn-tv;mxtIb@-4em{TWF#;^ zf#TnnScg*qOtS7L#Ab(IXgK0Egkk&UR^{c9UqOYY8D{2MX!{_cY6cxkdK;qrl~`Rd zviH<&m*cgn4WB_8EQc9Ew7eMj_=qvyy#z+|@uZBZ@bUe`k6-;aLTPnl0K4r-#JxcH z6@uT)y-HT&vz{2qbTZMm3%DK77o6OHSQZvk2vpYCH6H4zdo_^CKL1x`v#>P(iSfC7 zK2*8c&J8>>X`PV+(DalCHbP!Z3&!NZM%be)pdxDoQSquF0xZ7#b`sKdJ`j*|;KWk` zDdhN8;egm(Ht3K`)_!ae*+N?$j~SzU7IryyijT+fJVjxbTWpJ z6h4~~dOG%#M6jbx2~J?F*QW&A!3DO1Z{(tWE<_?Q@-_xlsnSWD<%Um|Mdk)bo)`!M zj7`|8fQ666;ytw^L1Db#iXNgBfoB`uW%yQEx0^lUxUwUibHA0(0E$ z)kC7-N1b6_L-PT3d&2(!Cr^)Qj=nk_MWb^G3YbJ)#v32Y_ix>DXJH1P4#5*9>S$y? z21?N398t+QD8gSvp1qc;2U_Vyz@OAv7W9`HbwC|l1!S+v4b#c>Oa{CbRzBOI1U7yD9k~W)o15onX5%Y=Rp<)^+1xh($BKs>0cs!`Ei-%$1^;M4yfoT4#R7d}w+7r0A zF_@ANbuzIZOK`9H=Mr z=pOLXfj@2?#pq7_Rn!K*BL{wG;%lybfjf$h4VBYh1DP8r_>g14bBt9-gA%F;8Y^QX z8v}QrqC6IM7oJjvVBl!@E4BzPD5`s>SbZ7~XA>4md`{(Unt7iln9Owl+>f1X@SUM` zeaAB2^TECXtNi3mZT^7hF4C;H8HPd&Ypeq)6CqbLbTSBz74ON1ZU?}SAN1rzs7M8% z{8kh&L<(P6Z>*|Y29~$|hX3->k2eiiun2R#2#lW-G$yl3BKB`Z(`&UXp!oO^elvKmv=0Uyi=vEh*3@BzCHkSNf{SCmI33)t}2uoyee zOZ#VWa(2EPvu=j4(#P!DHTY|qb!)E<5OTSMf5KBcE;rrGP1Y%o2oiYqcI{rsPpJ}R^_!CRW!xmS;#lk3dNLqy zhzCh~+?x*!w-x$P<=h03p3#2=z4NJa8qbp*WQE228$2A$76fdh!6L!NxuKW3u{(E^?A{{yANm5nIOGHt_GF- z@1{?Pwh5pc@mLOAu_pwAQT-|(s5pTmZjg8vRXN~cD9{iQWA=agPK#JVCeh!zpOgwW zh>^ua?K{{I(g+LHyw5+ULAm=>V;vzG^c-riM9Lmj9_>wr{RyMyn1liiCmfcM0P#0^ zR+sS$Zu1x&XeFd?gt6u{#am9wV|K0gPJ$j)6GQZXUEOL`J`=?{c6e6;+D%MpZ-4|W zuCaV*zPBp@uYK=Uz(XtWc|Bs0*Z0HD0E>Fw7?Q=6OLa~-PhH~9yuQV<_ZE4E zHSj~!;62rFFEx#be(%Ag;)C13H4v?U@zeNVh<|W$f+^1XpyZSIfp^1YS3%NU5S90A zM(gtrzzt9(-KNQtZurSH12o5j8cA$M^?QRO7%xz+C1+<+g2t}Ms)T2>ff+agKl<*y zowW`MWxzteb3#`=Jdn_S!7`^7dRRS#yc8Ji{?n{y-w5M9zd4Jm!gDIH$wyO@Q#6uH zhL1yd1s(6WpkVsxAc-ifX#o8&%TE2a%N^xXCI>p-_N&iTR=fwTWuycw@33rYZ#OL6 z{+RF~_so`jURkkk$@f0K@)EOEJ4?O_eVNew&ZRxXfoAvKC(yFqkv-F>(8E-`?dA0d~Oh)ye>VxAUv@1?fdlt zzi%%tC++=^&F4(1aKGk(x9Z~zZ#+BdwDaoeh zB=irq^V9t1mi9%}W+88#$iR+k*GV!fj@`|*tfBKnExctG8)+vr>7KFS_kcI}nN_#k zPMJY6zr)=zYYX}E4xNxRS6MfXml+Gyy_@|t)${BEwtxB=jwEE;41CmyB>u)X;V)YT z`F@*DMxTCZeYx@FyO`hf%=4>%?XokeJ^AkboJ6&m;Q2m#0(7x_GbG?f6u&EC4{YIs zx6nbsrQ%zAqrQnH9JF$`QO=tQ=?}kPVxIXye=dVDp-b(WvEQd8Eys>@HiP?_-)tZF zK7UUC-wDe|(LchnxqPQC>muiApg_ZRf0(D%-{UgmlWl<;M&%b(#SyX~KkWbf`pICP z*>=06?HkY2$NmV@x`kWr$kMVMd3<<8Y&wae)Tb#vUw7Z;+s;}}nN@~8u0_Fee8;y+ z)$fcboXbqP`@Yoo!|2X{rHdckr}g#>7C7%|?O#>2avgJT4jOk_zYi!oY_L%|Ci}cL zZAtC!{9aXZ*-kQ{^2g=39SO(8eG50X>fw&M+D3jT%0u+Fn;Mx{%57 zC;mBTM8C;91;37N%enPey&KW-nQuN1ULE?|xNsP86qPif26)f$Iir8$XU}RfI922T z*Y(H9Z5R5N0$w|c^B3xVZ`pb?EA-95*w5Z4XUi!uqUeXVL1u-p=Aw3v|5<~~xD_GM zVD%bJ008Jl;fqBueR5mhh&i;}of6rm-7RaDN;UZV?L9Fm6&kqByeKI7+5e@8>DMMy zR1EBRa!NM`xe3J6&?WE1v4FpenwNA~SjV`~iWXVsYlkLt&WS)c7^t`hY@|3|@(l5~ zV0x5XATKlN!d}DOi-5VLe!1kh><56Y{%}9pkPg#WZC9Q_AsiO#d%qKa0Hv{~Bl_=2 zzyZ_CW8X|LV9KRNJ(Z0iEns?nv|JQ12ntDdnY0!;+ZVw`f$!LNtoK;y#37k#P! z8xz7r;PKurhj%No{sf~s{`2%axZ&QjyJ|G^c{jEApKXD^ZW7*m0sewjj}2TD#p##> z@PUE3SceG!pStPB?*t%RhJuqes5Uiu=Zoy`{|XvlIo^P(vYS7yKq;LDH&JJJG87VXxGv(-Hr|>8Z-7T%00ZfeTPQ*Ih z7Eym-;2XT`KWSMDv{^Vgmqn}E^nc;x?~y4uM;!QPZyLyA-z*APdsqC2&X0Y?FyyWd ze3KR&aCZNqJ1PdJ{O6mTzzKnC?AtL-w*u3XAx_Apci~wd7!1hrvu=6!XKHl(Bogg@ zTEGcg{3IlB1k=QQ%qFqrP zJI?*{P-?_Er%juEZA0tkmMvjtJ$kM3@W)9Bx(i~o)G>IT&< zzv`3h3O3am?$OD?W>}3!PFtVKkSn(UuYVlCX`DfGpCf#Y$UnPN4dC#fG*mb`lVLJu zdCY|*+uNGDZw*SG81iv1#M9hzf6kn0YWVa3@pi2SBP=M}D}EbZX~%KtC>5N{vg3mB z7^;&+Rjzk=jKHY^R70=JXsEdrepjMOi0_bR_*8dia9x}>6wIMZnQ+bn%PZ)dDM&9( z5o$Q%1OhdK1nm=Xz zw2#0Cg~-ICm?D4ogR*~mw0tjyz!KUQ#Z-JS=YLKu1YqYDm^1kD zAEz-RIa*SDIs9lQF89>V9ren(f*xzn*M^W>EYd z`Q?AuBTn^u02IwlK~oBe?v;^mP*L9FJM4gjW%>`vej+kcCR(2SI}Xcj=0@(qX;l{|YeNGlMLS`{h0a?2j{u@msQW$T(;8^Z zC#!_!$3bCxVfV-LEUR7MN5n&4&xQUdy=se;IL414nsY)AnFQT(Z$r^5KW60O~(o52Fftw&f3Mf%@AGomVEQ`v6NY1GTtK4PnjA|II>T z!v(7Mm4w3}`~$*=lNu07>jmi z-Fwu?Ede?vvAqqb6}KNDQb0NXmpNABbOF<<2+g!AfoIouXH^tBW8f;$B_&8~FjLz9RmN<6SI}zzHjR1f6$QiX zQS0!;o|4UAmv8J#XYm4JVhDZ9lCK0XTqjL5Ii_6K1LD1S9@GVE5xB3@QyoI7e}Nw| zYbuu*y1$;BB)qa&U+{77t1w$Xu)Z=7{;2v-&_U4c%>Jw^SIC#`;s|9^ma8kGqz-lB z<2~+7+8u?x$lvlPtOt*n`+ZIibaY&XITQt*zVNc1yVH;OEIxUdiUF09yO4T?3eWSD z#-G;hm%F)5|E2>>*P|a#gR^?ax3TeYx}*eE%^zsuOL%Afp4^Ri#r>8FvswRM`R^*5SFH~vDN zx1>0{5$jbUr_cTD=#+GEy;)db%K;zHui#1We~3|Rk>i`>QdIVVW%N!DO;&U#xtn2_ z{D+Os(MHg7w;X#(MZxAgTMrG|W`R_^1zyK#?@PH}_k@yonirf^b+%-?SdbFrdQkTG z^No65OqRH_7L2DZ5_9qjKvdM%_saeDZR&1OC_Xqr6-#Jw-25jRB${GU5Y+DZ&R8srF@CU=(L}7V9x!K0cOsxy&if6Cxgh)XL zLry#J=d?G}S5)YDimN4Lp3LS%@OI&MyIy{BorGJwYq2zK>WuAQo3-QZp$juYADzIl zyZLwjr(fE*RmrqB}m z@=;&rV3mZ|mOyKgB&@D8n@&YKPbp_Ezh`S*pyoi;_NH=$o(sI|PRE&Q0b?zuDXrLc z);HFs)!lWo6p0tC|{D{MLMFsb5d`CSL%1dOp0e^b84##FIjg`w~xXt@Axc zFYcFP9Fgj$*m@w-2yT<&snbX95vTc^dLK`)`p6o1$B6I;d3CE$3MR`+#v7oH6;VQje~6HXuZENu@YTDxDhIHxzB zYYj#|aEUXQc*03?PFC3al(0Og^E2k5Q%gv0GjvT#gzuxE`X{!YmmCfihKhmv9#!nO zkKU2*G#xx5Ef+||RnP>~jq0blbDK3~tCoLE&QlLW<~zmN&z7cXA7_dhSNQ{ga{lNb zDnW~C>uD{avB?Y1L&rPxgs+b{Lr+{Ny>YpssSHD2%FmGYyaRCAEaJ?5Jh!Kna9+F~ zcX97KYPiu<6KylJHa=KOYQA>KKyeGm^iCJYO0EU&KW<(AswwP05}9%Q#5ofozf0`)}dfaj;$YoLA z%~2q^CUiLM&QF8>4s_4xH?nYzykK3;avl4cP=M7RF@2)Od z42LN0m$@9pQ=ef@w4DER!BfM!PVLz}0rlbHkulnQBZ$L$X2W7JNPy)4t)h=|N2yJ# zp#kjTtBqJuSHa)yM5sjK@2^_9{p+A(#9H&yudj?$ z#)}TW@?+_Jd|~t)Ur^y028C@qeh;O6T2X5;gUF`p+IL@VS)-caE%ODn^VvRyBIg~r z8L7O^)-C#D%M9eS_qg1q92AIy>Y>jY3O&S&l7E9<(Wfbef%9pmr(<H zT{WdEXy1utEJ-2J*w@1YTJ~kjoixkv)K?hX!4r-#d8bh~fz+mfkx9KS&t8+KSFxg2 z%lh)UI9}E+)fP!ij&g3`ejv-(l@3f5A?Bqwr%2*d=q1iZ{IrwG6;GV)6TUX^s)_@i ztjEnr%yl;4co+COye(EHUuPhiVSnk(A$(M!E#j2E;L?pt8HpvGwXbuPCjy7ws!X>O zjI2S)CFN&bvu=)g?&q(1E4y?fV;3A0Ca=yC7Ox{6HbW)~Ol5z$?+m9z&3Wd;l=KD| zPRjY97;!RQ<>O9K-AGzEPd(>UX#H4iB69-U_3vb5CIyOY$yBC^i?&UFzIqgL9#Rk;ZzgCz z#xDn)i+^=~?WskW&T4d{tx-LuRmp`=*YYEvVJT+%e2*QpUm#!Jj(gQ-exsMHOzLe{ zT`tY-xMh52juDtqX!dt;Tb}J~<#_EYI2hamj;d)pC{G6&s&~^tJU>uE=Zaj(>iIby z)rC_KO`L{kqP%eCLf1=L*~P!&A?=xGiNtqkZx>!x|4M{`jJpe4@_^==x`^&+i)OPx z)@v3d=WVsp35BlpExuY6xi4{&Yph1N{}6?+j059Xkw|W#@{*!qsx6*+#nSz{bb5|w zM`le-KdMyRc3_)2BL98bEKarR$5EJ4MrrKrps6OuYy>)C~lrb2fd{! z(C?i()EKBINoAhDg`g{6Z|PQ01}H?woR$-w9%Ubk5#zU?dQqYDhH<%pzV>g{!5%xg ztxU#1Zco4XPYkm+$LCd+YW<^`CX||W%`LG)rf=doG^<=!+s@|@j}Mx6wSovJyf&i4MRy8gdy~E|DUfx;dVZR1 z9=;1RL#6LS+xy+)tDa_V$$+ltT=t<*bktnQzEW!3Mye_lpb(O4=x~9%B#VD1TDp2KeU#k99Pk}rZeN-_^5&#y8^TH|UCr{9KGf;g(i7k1=ANF>N4h1EZ8yNu|u zYl;wPOLLxSCR+bq8_vhM^Q8cq?Rbnkl)m?=)>qqvkLMhWUJGGTg|@Z&J9=aC74a8G zPc(VSFgzVIRNsb(hRCn(o7KZAFK~KumIKSBH%@)}>)!Bgs zF)s_~NABxW8>nhwYe@mj7OT3*fOkE|*j+YqTjzA6*!*gPnknd~i%;X(l^7G|&E@x% z(O?=AXW!-CBU`>050RtMWuRwuejW03@fZPbCj&ZMZY2x;)I44QqyEg;#*VWa@Qx*M z<_0Q@@N+(wD#Pmsz?YrHkq{qUfDSp82!3?git8IDyt`{@eC#B3JQd#^S+=oz7t)$_ z0u;F354~y4x3cbr)J;lR(#5|dt9&Ti%((h{eZ>S+deC!or{KKreX9^Ix)9aGAiscE z1`>9>I@J%~#@~w&9h0>GLM3;lxEd2+G9I`1uC=9d$>Q_e8mn(FDfg>blY2Bvlh5zD zzL_=7O8i?u!}`9u?g{vcTCC9O$Gsd4IE7ujR7m%J=(vD>-D@svx;OgfGwT<8 zknYvUmGAuNyWQ}>E%+E?3?=S^iO>CgA3C}?!6yAj~0Mh4&Iwao)zk9 zloG&w&N5lSs062AED~4SM;H(!pHSSLls-W9m3XAT$|My<@BOz5_UV3&I}J>JHJn9F z!e(zy3O|~0)JOS#>~|>{6TQSXb`inK0gf^Yj16Rdg}afgeahGqKMm+n+_5!-$4-cEX>v zEb~q=PkfAcCX|rov5_>Umjo(p4m(rPB^O9B?cAYsm3qAw4IotGc*m1L2{+7MfK((e z0TvGY_XUIhS=>U^^j!Q2q{eT^$TSh_ok% z5+c2yzb8T4U|64uKCFK}BB40X*Yd5tu1> z8=O$%(WGK`MB88${TlxZDG@1Lzq>(vBXbqlt@N?P*VXNH_xf8(PcNCE7@ zV1K*$fCmLE<3konMy`Ah5X1Ie^!ba^4Q@&KNw;2xIGp!h4S=>vO+*1SYH$_oce$4i zte3QW&6my5x1wo;`$83VqI1w>Y(Vw~G`_Uv;J@99{|8`iW=}#WFWXs#AW9x;vYvQ; z@1mFiN}9$apGi*%t_(g`Ug*{Iuv@rgY5akbqEX-@08F@0ry^i@jm2EPv2c}E$k@9! zUCq95_`GQ~NZEoCa_qmr`H#7Sl*HoADx)AnM5GT}Xty>`o#zAhk_&s1ixW8KQ&qFc z0v4c)Js!T-FwwwX^v_V%Uo#cdku(p8waYm@=MR3jQ%;LOhu1u!Z~|}AtSuUnxT-UV zatT0`c|gGM2P)gQfX234`Z!%YMev3mkGZp6ihWfh6xTa}wtGAEH-ztBP?Zi8gem*M z_RagQe*lY6-CLqtjnE6nXl!F%6&hn3r3(9tG;-HgY$SHXZ~ox{wV@-7Gx;L1fQQ!x zCh<&eE4KIV7bChK!rR-U>l6R2e$2ZjLhb2wIo2*N+*I?9I2)jZpGkACaUhfrQ3Xg9 zY92y31^x*iQxq>Tq_VR_+myF^QlOReOWl(nkVX_Vx5bMl5=~K&2VK+!ltk~z5WhL& z{`uq|NCwDFia@@qXX*AtT>;9f?hQE79A(%3_Fzbi*b4Ln7RO?~O3!K#&j!S$E{AE&2AIr)Vyc2%8h z<8>R|K(PM;#xD_dO@ex>%IjA!zxz`TQ^;pUpD#0>>{NYerhIpPu?o_z6J40W6bXof zA8x>G=qQP;F-mhpVgjRLklYeS9Q#uT?hPFb21JDnAmzjAN4&Nv1mEMtZyGlBU|dC* zu~%r{B~6VW=%-OHws6Uv13JJ*+PKWLLizxO?Uw`dYV}souR5M0jwY-3yldORoWVVn zR_Tp^!b?k%;wa`n0c^iey&a5UCiK_x;b+Q(8zv!Bf>AU~!N+$jI_aUD8 zCvJ-A(BF`{z1$wTz-vI{m3k~-0qC6qg?E>1nKPudl1bc&!tNV^W>#HP27{Cdv%#0+%u;zMK&H zVVO9c3dTjBsu~(eH_m?oAf^e`I#$J~gmu@f^Ofo?$SVjj){_XNtro5ilN06q4nN>H zBaGVh(D-`yN$+pPS}$*}L0kyrbVkH=P2Q~y6|9YSs6CF*{`28n&n|TjZJ3oqacyC9 z9XJ<>P&;|3nz&wc|2Di-)7&@w=!yRRv(($he8f1_=^6e+FYqPAaVIs3Ur;a+-3{sk zFQLgXQp2<653~bYH2FczE2r97V2LmafrI`$^DAC_GWW1RgcxJpytNOwka)on-K(xJ zUD^s1)H9Sn4j@Xhh`PRz>>L!_%*g;7>o7|dJOasTGS0cF{7h|<3I%FpoB>c@dbnT+ zA39XA%QE;!jpfPcJWyIbBQVLmSDT5b?tMyZG28Yz%ni{01EtlTO{hTez_ge%`=Y2a zt~UZh$*0DMF(B&SEiUwi`Rke+cYOdXD!-vosqm9|)y$Z%DIQ!%&3~8=?l@)3uPG1K z9wgHOSX>z$*T6mds|=f11IxZyu3&pui)j@#;X8uY7tZ^tE8}slOjO^92mB{FVe^r^ z-Hm0yd*$qodP4`I#~&L#du?T^o60pZh2OXfAXBT!rjl%8kGg_#lt*^9e_jO4WD?XE z)lUI7|B`qX$pz5o6tYm$qtyvLiW5js^4i|U_D~fb)wh?zmb=0Ad;YV6S04|sGl@`e zP=wiSc@Xsu2#vg*KyC@bNL=a>u7odwh4@av2#a@H^4el$m4 z&vpa2AESm&k$XV#Tgy1VCCyNW``LB&ZTCUUzR1lw&8?qwn}TgRUR|Af61D=e0X*J4 zg82I25LyKU(_kG!Ih;e7z*YSY$2jcN?>Gy&Ff#16A}4g@43QclM7Gz$_z4(PX}EYS zEiS;UFf2?DOKsi+hJQYA4ppC19N7ck`*U-21qt$VBeDC%lsRedE=Y3s+rNU$diP6Q z?n4qwGVC$7iq*8F#3LpD4Q|;ylt=}t-Sum&&4Xz(KID3xS}GV6D?`<;M`J`!)rpYw z0}=C}l!Uvp-maJZOMr_gG^n5kg*7%LYH2XscM?HLb1c$vevmo)$OMnE5QNd|(Fg`1| zmfB1aPz8a%fWoU^znBxzd$y#6pK9^A&q6f3Cfj5;U!l;pYWWiHD(wi7nv~Vf1FdF=21d;gCA3kbm1XQsomLFwTnNOz4zHGT;&s4}}1O zrNYCX7?#HFhvFl)MIAm1SAgO5yi;i1dvR(Uf&Pn|HOoj;Slmt=A0kDL zRfemR)5^rnkh+5}{{=(@38;%1#58(nibo!}A~WM*(*MhA6n;qw(mERZ7t(pZ*OelBcyEy+)vmjZ60uim#bY7ej5}*=}d^?2=qu52oYwR zZd(US^}yZ!w&!*-mPLsbIKI|A#ATiYS%ibVg)3DbbPV!OFi(dd1wGQZqt+B((%;}L zn}e2v92iVFY5)+ZI)9xWEaw7RHNnl?7uaAzU9sQRc8YgC)We-593*h~7 zflps%2>u|2W1cZ*JgvBH{|&`+ALp5w1H?b;)L`j?6~r{YYbL<>SP0Nholl8MpN3We*` zC1T-IBmg22sGJ)@9ToR(Z|f8gDs|L6-j!<0g}k&fTC1taszD{LPKew%fzH@HlLfO0 zuz=b-TK0nyfiOCf7p^0x6}zUZ43}X8JbsdY2W`@m*g#`d;O)^qnqNAz^?GyO|y^sq%6gX6U5=%#K$Ft->9P&qYNwuv@MHHWTt?Goj)y&%Ot^UxJ-AP|&cb zb}Z~44br+?g|LlZw4Yp)dkO(2YZ^cpL&M=nYMB-PGq(aO&y4&P@LdNvj(1mItX$ZH zllup^d7+DkbP*n*GaSZwMdRqQm9N8`r`Y4^n_*9a88W zG8Zp-lQ9e*2+Pn|g23%y3T6M%peMfiSqq)R^d4VGqjEN=*b#-c^AygGL!iF6u(JR)^h?SO0{wb-Gt8rGK7yO& zSltDghZ`K2raDA(dqvtoY@T8jG1O!|XkefOoq-B6x zq&_7d_;|aG%MgTCF7|&;wRy%7A``>AFu3pQ>_?xWou^!dAl&tzqr}aCsM%HWkeEOK zaN@Ro(t6IFt|09?lI z)5+}UHiE^P5%}7vo+gCn);AmIjnIzQ2q;X=W52i5B!Sd(Xed*k$mbIX45>;1yS)ja zK)>RbIp#dhT+QFzSv=PB@-ATEdLhL-q_TsX(jW;mcfd$Edrd7rNM~fje z6;aTgCRYJoq+|??bClP)CQs43M7y=+^k$+0vEn+}-sr z_C<|In+T7?j=a7_6*_~LfY?EE1RulTiow0k-jdbXpn{qM%C+Y5yJ-zpc7g)4f$4p!3uDj;VhFSqssGtwL)<;(4<-;Yf9D?}@fYSwRv zI8!`m%~Ni0R%gigH3W#26d-+(2EZIW1{f5TJ$_xZr-X*j#v{WcZ}Tw&p!lIzics?k z@|Y}fSN1?v^ixFp!lmVlUTa*$pHstz8FMQ}%jFMWg94QY4dlp&3|AHWp0Jh3^;Y7G z9l$0w719||jS_cTv9+56q%gnNA8rs@S=1rsW-l4HQGuY%2sdK3uD#7&9p;&i(Tl!r z487FmkSNKi129Z`sA>dHU9y$tcbyi{31zs==pp6e%YG>SH8M{`sr7#FVj+-4J?GY~ z!;p}aV$jF!fSTppr_)$%bHc--vv{Pqj;GLU&{YsLLU3lm_ULcA*=4GMxN-#oP11~vZB{!G-?mU~_XZW85t383<&Ss~e-bK`wKDDQg z`4vnW18A3|-jeQiq%no4s;9Th4fyNuV3%vOpzls9n-P~dfAHe92Qx$t$o>y7TW64G z%Mj$@Iu7zS^0Oy-2vo8pr)|Thn*9za6_THK7|y)|-JD#s@+hlLLQm;HzR`qzv+usr z@bB+G&in0YY+P{ZoqVcYVtjt}B;H8~6^9OW2w;)$P#!`J8Nsv6r4zU`NU%=<8aITi zBs4N1M!{1y@=P>KkG21 zj>Hy3I49~bH!m_@QrSo0LzLHEV3$t^_ri4R2kR@r`-1D#*mdmo@r4IB7@?7t7n^2Pdm2=X#;HgNw(_5oV^F6dX_=nthu|Lpxw zrfs=?unv;X(Du#$l@PxXmtnj0@zH^de)S4v_Aa78@LGLj38F$ z?=J(=PL!{>LMFyYj!I6pBKA#$F*yl~?1NAR3~e4Zh0I-NxSoCO ziAYVy`M|3Kvp4dthbwcm2)1a_hEs6d3b%=9U?*XE8|rYdYwPDUJ-J|?6<6OdKE2&| zOWoDY)%C0R&+(pV?-g-=@*YSxT0;t&pC2j&lPq!ysvKT>zuJ32G;RW{c5BE&(+xHI z`N6HajX(4nK(%E#8!z2~?R&()Jh~vK^T9KthB`$BV9p_u3J>2PTDR2*sVUSTSYBsc zCkR;$AJLJErTDP5`S9n&piuqp68p_&0wB5@!e#OmWjq0R%=0t`KNF3Wb^bXsU5aKQ zL+r=%vOI#%^j)&xs3CtS%;k*kg^>d12rh+#KONLL?-+2%1(xY(ro43P!R|uJ-Aq1! zq+g0-SLc=OhYJoBeh9m!VS-i<(4~mC3)C{KAyA+XRS4+JX4;&+0{0r+=rh-5ku zXemSPLjO)4HnDm(;03an#&+7p*rGBtF4UVQ1TxU!SIILxyQ(nTIWZa2yEDSHYRQv< z1|y?{b2>DZSb#eJWTSr{%rum`hA_`PkOH)Yy3@THzvdYg+TR*<-%5YP!6e2eN&#kKqp5cOS;B}KL(kt7r~ z1=Zv!e&Zd+CXL-RV-Dk|Dr0|_mA%1reH(&drd0%j`U<&Mf&#&JniG(t><51bS95d zd2KK+@a&la77EhrFGJ}iM-LL8k7P6MkHG-S$i|297Go8+KN+MWGzyfMKQTeA>(Jq! z1aDFLywT=a_=z=`-f}Sl+dGuN06*orM8Jzvz*_Lk#5~5}f%JUGRt$Podf}NZ(HvCp zr+$uw!$izE`qfNs68inU&tB_FnrfsKg`*u?0LYid3pedn zUTxo62ZuWq)~E~C;&ec0PUK-V(fKpL-+3-0IEPQ~@qY0pWm)e!uu?CN87cQ$|DgL( zNQM_hx=22;!6c1iMB5kW=4XWwf{`3Xq?As$&J*37uNhL~hwdNVwb=n`aWY5LD1;_khs349z>E%*zmT(8U4}eX0Q-B; zTG5SZZyY*Ha$gp#C+W2+bXC4*!gD~9OG-hdj5bfM)hQSU(UKGEG7{Ym^6twxi+cHe zs2T-&nS>lL|4& z0JAyXc3)MPdxQy66ZsVUx?4e5&`$Xm5Ws3sd`SF#)l!wgF6A$H))a&oaHeG|RCgd( zPN{O_`^RR#Cy$WG?1Hh(AOsD>ztW|ERSskzGG^$oJao7jcYOD(OO(nGW`BFOxCSo| z96M`flF=v3tHbZ&3uEQ7VfF>&b%=r!0oinHt_Na$JD3+pybg7cR8&rfP)y)=HA{_L zeStm?s1RH3^q|ljq0D-rRlx$9IfxPc&EysBJo97uxfn`0oxkEgYT`uqH-_wVT5>vI zn4oP1MFXXa4mF`DT*=xMyzjw6nnT`D@0uSJsFQ&;Jqcx)*OopEsy_yqdST3*ZpBJJ z>UE=pF#s~2Y7EzTVD|!sogVF8W8Cibt<|nB17O4yIO6igO@!cXnd0_KO?3FHDUWpN0om;-m?}lTbuvl3$Il#%_lNL&6+XYERzrBky{T zkwh3^2EPh3o>soa-bp8224x`a?m&kaBPWA_L{9Sl;${7sdVo=fh)<9iiCb8CCSzQL zrk#gcCDa(+Tf(-BWK;Qz{v5S37(j@$bhjeAU?gdKf{jM*>&F-IW7V*CoTBIVJAM-c zAWJzO;GVl=Fri@s6%+w*8-ynA0OaB4>1Uhu#a0IR1OgEdMO9zZ1X(ZEdZLR{;DJZ3 zlM8k;TRFFNp2yQ;$1N9x4Syxc-s7P`Etg;b3-@JpVu37>jdCX&%7#Q(pqkP#^DcNk zhW8@qP=hDrpf1w~!3(^?4={x0lwy&%kPO3P+@Y2g;*h`V??3W!AaNN~;hq(L`)(-n znCs}?4mmNL&bb5re|x)53!teGX&rm1YxNBj#{U!4)SF7SyrU%eEsOolzl!rKj-q{= z1{U+7$56-%bG+JMPcTLx*5* zw-RFSR5F^9xDuf{+zHur~p_v^YT`r{VWbgeE{6#9e4ZcW!>d+Y!_Ghc%ey`Z4 zxft2Axh+0!d{ln3wEJ}g(?p<`pmd4KFPS`C0jcz1e1bfe80aNd6 z^+GP3!?VIIdfuzeV$|RXd@YdM^~cbnFsvpPyR{1wcr?An|6PsR+)s+o503l7mAXP# zv{MY>r8r1G#$Z2&yUz<flw1B37&nLuqj*lF#6KCUC$|Wk#!OBZj!y^i$iOi$w%YEDq{ha7h z?TCuA+p|`bhq?l!bO_%7y7oW(6n8ieml^Q8f z$lFk|NNt^^8cq@z0ShP`)=C(DEry676$O1gMmdZ?;e#*0&+IxE3s`}UQ-^nYHfn3J zFchDk#{`u`v5pe^eB{UAZ>j*wgQ~!|l_Fq#FTuUqJcQ{i6~cp)Bmix;K6a;*Ji@nl zjyza=O7PaFj&W5hB=#-*@M8q;?-rClhT0kmM`32|WL~rgJU2V`6w#|wbF^4M!s?A> zYGU^#=z&i~5XYX_Mr#&^pL8C-UYvluWfm%ybaQud1>w(00*CDKZ<*_$WMfD?h>JbY0-B-s;=K&9pxN}fJbNjCxOzlCm z(r1l^9|@aHv)w%ws{wuldTf>)LOvKZZOH_S!Pcs#s{!X9#W;{X6TN-z(plhxVeJV^ zA)Qa1mS&BBcDLLjgafw^-#W`OJ^!i7XTF~KG(8@0*GRUeG0=CjsGw2C*oF9(-@OiNJb~7@Ois0eyxx`Z_uueC|F8fCp9;gVQ;u0+1feqa zl*G|6x3%@cNM%dqAD#u)Cuv3A!)z4{M?w75Q^hHb-4TB62x&LR$&U)%pCu>VN4;(N zY{3-iEBZ@TCv4DY)^RCNLGl&mIh)pj{OsLusv(>+a^x(m2EJfnAeACs>jNI)H{e<( zy53~^tt>^a4>|ktVjt3FF`jlu<;)ZqF?7{A6fImkgSLgBvr!gzLD$_~QaSXTX zXnsoZ+i;wYCM0J|24$nRmMzSmk96%t&H8dnio^PNYwrg~mJ;g9F;&A&An3Cy`xKuk zhUbt4ys}~&drkmP45Q~(^XKrLs(VcF`Zypw&qtXtNvOy=A08?g{CXmE?FB+)c z)ONY_8Fl%Bx-82Z%j4wb8Np5X5O`c=Sc|tYCKR_eE|_Ai2_QFdow^H5J3MLM6bAeBD=Wp9cy|1en`T)2}id#@#*m z0wyx5!?VVdHc7-}?j(o01{CER8{KPG9qXlNl9;bH665|5FG1XK_(N>)hpeo74@TOc z&=0W^KjdoUc0x6o3EX9H*H7N#8)grV?UT7WKO#QydTxeG~u!A_CjBGoXt_Fhf`EzadJ zZjWv5FthXXEkl{fh16YX3Wv+f4jnlVVZfe`&xbDQ!7WzI1QF5Gug>~tdjS=}5{+;z zl#ivtQuko*1}z^r%$0iL=?o9VC7f8ZBUG8LWQq0DYG_v-;?%xc+l=wrP(T=Q?7^_k z?{b2Jp-+g6kNLe9pN>_7KQ>~-2NMJ#Rp2uE=mbN|4cwVfqInhY;i>j;B^%?4F4>Z3 z-Se3nhN{1o!*bue0!ILCCx2i?oUvMkR(DX< zut{a<^)R2ji7r!mjFrDK9m6ax$ZcOpv`M!ENsdOQD3MT~OEGn_^3=fNn3H(VLNYH1 zikB(>BDkUQrcuZcT*? ze|m8&^}5SQsF*f$_m?A0p%U}6-z82FODPqKaGgrb>T&^BEn(JRsYIc`%_ga-b39}uL$AR)z#aXo!Ht}&1 z>Q(nO!$av_Otjhka4D{NE1gY=r%2)5^+9mY51+4yHh*Uh>@#rVOuIqBFSUccWL1qm zbby@$!b2)ZJl`bvd^vKMAeaE^AMM@%Jb02krwLFS`j=Hqk*`e4hu9{@b`r#_??06; zrpuc!#eNGZe(7v(qrod*b>?qp`}p?`O5)Gu)qE28bhp}55~fd}{_2D>Nj!Bu0i^%F ze9Z*>3T!ISz-gza%CfNu*B$l(-mIX{qaZ0_f@@PLF!dB9!1A4gf+p7=W`>x^QGfW5 z`S57keQ!$lF|i5hTneY<+E@Lgc#@>=52jZSjJ;OkB6};^{$AjuP7Wru%v;kXhGf-+5(E*$c2e048GEJywj(v-<#@ksFhZ)AXr}jY= z%55;}yT3yvK3`zV)PFDdxce4{dumLizhvP0dTWrh*E75~3nvBrdl=uF;O+R)|80II z$>2RbJNK$@=I@n@tjET^ByURDi9*ROH}@~gYRy?Rpd553?{8hkCj74n9uAu&djlh0 zs15o-t(^wZ`9-0V!uu?(ll}Wx*Ne*Q!BmQl{@8(8^8-|uGkN~ zW0x)=6t2yiMBL~gaPF}`dbnjR8nfoNbUyD$k>70g*ndZ3|2>xj`^v*x-SR#K!45KO z1h~D&a+;O$YIZIDu!Bv^or9Vd9@IuJx8`xSQlViOYqzdW5U? z=5I0Ym6H9jqX8@=*PF>#ylk`1L@CQiz}5M~sWpo(y!zLC*Jw1!mU@=4*mDI!(U-2I zz)sARni{xEi&Zj>5G+EpBduhi9`7AnZ16%3X-iJ#Z_ zaC*q)R)5m%-s&HK(iu4WN&cAlrNMzyiCUSF_sqr>JH#;zRYCdp4S!<@H(TUUap*_G zvc_hukAVp9589l~A%@m)!5!P!g-3sF)B9jp1hr4kQtjF(1De{Uw$6&xq&AUXKVQm9 zwQMZ7WzC&^-kY$K6%C32z+|d;;#5J4Bf(yh|67{xiT2T(p`sekcJQVO)z9C3_TBY* zF7uI$1J%1EZn9~MlkI}|RniPo7a!Q9zm>dk<_2NIZdo}=G5KA7d|A2awmNn9MOt?8+|j17C^`rqKLU%|I(lS6 zK0;cjCX7gUz{W;BH~O1F*=nxSB+Wru~lwD2t{Xo)p`QsdGWUd=G%MRT> zPGd#k)QVY_Vxm1OdCyt=F}C=v1fbDz`XBr|tru_0f9GAal%w{#^=(4(`>saA%dw@K zYf}05wqCVWz-WN`icXVRq(#f zVx@w?gJb6gM7+vpa%E|K>_ghpm99L`F`sNvf+}^+bKC$N0=ykYU74u;YMDmj&+~^l zZYvmM^V4-7)i_xDrv}(dw|-ibk~Pww!d8h;w0Gle*yn;ieVK<*TNinKerN8lRx}d@ z|E!(Es5KIPvX0E49)uW--oxJ9GA48@-p-gb3-iz)0Ej8-5P+ z!%pgVq9uiYFBd&z>W59+%V6@#xh65HDXrnux8r3MpT+L2tw%t!J%E;b;>=22XaOuT*Xb27=6M0F5ht-y5)M?Q&~4}93MoUizWWyFuvQ;1%uz^ z$716#o8v6M*toweR%>S}gpV$0mH1wJUBb?BkJMF$fd+s0qQ11@e$HQNzM1PEOIWFw zjfEC(C_^m~`VhxZ2TlzMemcqZLMlqToWQ6!8S3~jv4x6c=F=ITGS$I!TqD79v@|!P z_c2&;AT{0&6CH((tzrFd>$QFv{V-vj7aVS-TMv)z64y!yUEPyj>};J>qGOyrA5fH> z>u@bMaL1BsaZ%Uq4LMuoik=9D`$z*%PbOS+ZbjFJ#)6=$~%+gYn9%C$k^<*eZ>m7sO}I3(A< z*G4R8%1Y4{P~2Z02p3!HvvOWoqRz)S)nw-dlQ!5#Df6*gmvcEN1V&LfmA)Y7iNH?6 za=;%pei)KcA(FekSgT5t#J(L)xXAx5TyVZGjc4kNf$nqyg^!`f4#S#Od)6@c8CG~} zeS}_%mMVjf{ELQ8baisS!BM~k-MCYFV^vEwkIoPY>AbFN&-1MxO|s$oBI`zIWx)l_ zTF+K(ENkbP7PXWgZJ<;Gp`ZrrJQo38tH4He9&5R@8|@V_pW}KXE7}rf`7PgP2eMl; zFSDq_VU)lZ;CEogAm&DVl(;}w~dx3#=^heJM0(4oX6i|2Hzv>k_O|ucM0Nb^&hv2S-&W1Dphb$ zEW}Mqj_34?g;*ClFFC0(oyJ~Q$^P#+G=9?hPzv$uN-Jcp?+ghZ#`{4E;w-4=DfE>4 zZkB8eIG5ky?x!4MD&<@e?GJ577zB@jut{}O1nyFiLp#xxs|oV#Pd zsNfUxmt@yqQ|G!p2V#4oitpZvTQiq7rNc>v>^?O=GMg?sn(Ma8)h8l^KdlS6JcEfQ zR@|n(wU64~s1j(Yye8uLKwoB}=^)MGEW^P4Ue|%m`$U$(?RLc$vq3Vv1o!GwRG0Hu z>ohK4^Q_dPV^%s_MGk5ufwHt=Vq3`td=>9m88EqcT!|7n8wA~%6TA( zUG2BJ@!*?LJ$G^_e7TZzw$ez^l`vi}6Ul8FlGIcc&2v#^;C`p&*_UMf+%6_~s^&s< zqki{M-_;8W58Me@f?$DLhe|i7rn^i+Xz4Z~YUD?k4G8fD^jW)NE|F8-Yh4&2XOEnV zye@k*t-?Kbz4YOhZ{?TKuF_`4rIO~&2bDR>we1_EifWN8Jz8sbUs<(U#c3*Y+X)7| zuAZ#=PvaB( zmV)-?xv8M`<06rD(oB&Q)=FW*FO$AzU;M36p0+Az{Co1A57*q7-R&PPUg}J%H434- zwIml%ljAuDs)p^d#?iPKveHpwo70 zr#`G2bdGCAHhP3yAt>=`Q4D1eENAv{9dBQWI?;aqD`lc+;EXSN@=g zEVbWEi1pFb3f(tJ8h#U`%6kkX!V=06S=Ql{U@7h^OMR3m6;>M)Z@1&9U3ieJDK4J5 zkpu^pRU0(JH5@KK?!~>VR0%r5H7pxKOY^fJZ)1Us;-1!L&sX=K<`-NnCv($nY1Vfv zQ~a2DZYVQJ)uyyy*bzUcHR4$r0lnMli>Z#XVGpHT@F&)1KD9@GbC0Eln`BYJo@-9G zoVM)`8GQNgj)l^TWkGbjgfPS&1EqG0pp!!P^+p-WD|9tiFvQI!3a{)+LrvRgPKpuy z!iV!DLDh6VOY`#z)3QL6onaq=fP-_aM|nKS(5-X|hgdyR0FjCj)UdAcX~H`$`Bo4S*KBa~ zmLb!rCp-Z~45sB}&m%J{9PF~T*x!YVRS`sIe4A_w=bvvh;|wcr(#o_HIL!5KSnNmf zz!q{PXEUW(=T6OtuhhqxdJVYEP z*r6 zBYj6CD)wOLd3)M!FFOf)SA4_S$a_s6;-O9gwF=g`sO2|ag$ZS{<4q**-JY!KgLxm= zW;PD=&8}mLiXwCTJus0p(sC+`AlN0|bNQ*>Y$cwm)(cP=`SbuLfh<&1(uH?ZD%@)% zT-Y^SYs2zr`U}jU=SA+mqNq>bD|E?UCCSt4^el~(8O&ywmdD*1E$SenG<;g02{DlP?GKTob>Z9$hQ$~`<0&$^Q`bt5 z@sBjHVT`&wXUDu$`3hP>z$G%!5;^tw-HqZsOF5eFk8|1W4Cm{IGcR+$)qPTsYWjn9 z-He~`no03+M1kQ;iwR{tR33u+u6Q(RO>U|)#}yBo|Cd%DU78if9qPdzm7SgtzzupAp0Fx zpJmg`%TK3gUfAe)sIQakFXfV^oqKcsLjG7-k}ijq(l>5GIBX3G0dMt0e8hk6j7UvM zmG&%bl#>SS=y(?W-7}YsVe4-ft9|l*SkC5X(8r^mU%7(5)lBKSyspc?dokzTl8^U7 zlZ(I&_0*URDq6=m&zzegEhSo(sG7+9fykcHJt)-9;Ha-1mFDUt6Y*ck)8`^=wN;yb zREs#m8uU}S6qAI{avMTtrJDeiLrp+fSf`fXOL?6jt%`QBprd+)#}YL3V2DupR`S#&>bI1wr!8%fAU+$&ks7-P#=RMn_JIt9E zXAxfEF)=L}UFMQ@^@cu3r^@*5=u0G%2jDST&Tu ziI$bFd3bNMD7tw)lcuP9UF*K6)|tA;i#b{>Nz5@n2Mm8r$jcV>^H7%`mTF?6|nE*ecerlVs^GAQ@`W4l% z;+}LVNw&P^powI~TbDQUo$I=iMU9Lca~l|;NpkP;URNf^4TusUHuK&FEX_aDC-zkzZD&l8}3}aGPwQSkPzpL`|CL@1tsjo za$Zc|^6c_f91Eqf$u*;s=tbMjs7{WGK9UU$%iVcC?auwKCo zU)yDh?88dYf zk{Q{U9?00lv9Tx(iJ!eE{)<4@Pq((+^JZAb+RP^(^fPGe(#bctT+W`T(f+(ARZ-eh zH7=4)O)T5uV5k7q)m(iD%cPIdwFItAQs47x)(1z#VudgBJpPt9NR^vjGfJ-NhvjP@ zDtDBgO^-`oOfaXcOa690DeqQvzkJm#vcv5YVHV{#ZpT??ET$p{ zz;hCVR!%6S3XZs|s5$z5!7ZL8Wk(zel19&1%Tm8f`^3q{-k?#joV9Kv*x$5va}vU% zVhMD0lT=)=!$ZT#MM;I4o%mHzLn>UYmePe~ZrVF3C~0ZE^qoM-V+KNV7Nr{X7q<#J zZt{R*%0|ItGv)Iv6SEZTUm%RM5C^PzGgOI=V!2kIG6v{UOPIAs@ z-r&7|hLP=6QN6HvEo&$vKQ;Y_+=~Djml~yt#p36}T+ljZ8;*KDoB+C;v=&w5U3qRu zSit7vlYC8!Z#8v1g{kfoC*(W{&97%B(y8TmoGok=oIvmv++KDfv(?K53Y53yTITz; zU-J;f)>zWh>C4qI8eMMAzX#jJF-W&hp#%*jUajgoy_CQ<>Kp;^ly|}GsFkN~HaEwo z+K-+FM;{=73A!q?FwqQ+``j;yjLg|A5-J*Q)>Ni|i6pi&N`8hiSv1ul;#Ak?JbC9* zFQU0QMTWW+>VSEo37U3{-LDw(E(|?i9nIVPhA0&7VH-dHs=_O^Vdb8WV|TS{nwGvm}mfqj* zPvBo2bYUI!)JNO~q>`~KYFAZxCwnaf%Wkqy2JLK^q|}P;OMnU+J0@FYJu`s!9r2w#jw2M zoV1X(3X~=B<{$1@y;jQx{cZe^Y{ju~bQQ85Tk6jt)xJul($$_)EPU%Iq;$FUBxd4w zzvfJaiWU@7&M#E(FB@(Yu~gYN-{#5%k`M>3sjhG&r4WrjwFpiE7JczE9bVJDc6uJu zV%4fA(A>t$qx|iSab?$D^Si%5a=x!13$X2Up>~{=n>kR-M@pVhp8>r5 zZP$&_yLyg$$Plgyzt`MKH5%;86OG*eZ zK?d)|*VnZFJ^f+Idq#eDi*AKzax9Ml=JX6$6fOtvaFF3J&jMhDRj13|-8ndcs&HE? ztqDLC^5L47KH?47J?k#XGpmo7d+EM8<_XL-)%}ta_AKE^fW&<}=ro2KCo>B^*GpA% zNDx1A0AX^HSW=P3BJRKuG6Et44SGUB<_I%%;;ytvwj%inGQ|L#e-6|56B<`kt_@s* zvJ-P3K<}cy`syD77P{Rn4an**d+*x(ofR&mqbfvq75%*sHLJcb@?U0YLvJI%-nxre zI1s=c<|SSozWBYK8v*iU72vWwHRo9oy&H5iAzm_Hz@%qUNyOrpogL9DbQocNg9R!? z19JF$iJ!DkV>c_5<H4gjRHSTC{%$nxOdS$M#VH(#<3@4hMj8+?d_9|g?r-2tc= zE(<5H^hQ%*cNqR0%-FkfK^fJQFeZN{qUnS`w)=9?=pe;`y5_BmzUVs8&^033L9{QK z@9YV@cSLLuJupDCe>U?GYP<#=-`&({IOpxtyZZrFgo^0M`51qBG$br7b?Pw7h& zh#)x!(HXF<#HWX!mEiEmCc$6$3Vi+93+bE#a1;PAr}?|Bl05-PbN*XE@;g(1?EsgC zzub9GkwF|noCm(0$Es0+Swb%X)JekEYRU1_tWA3Qo-MgG&JJnP5d4~a?h<(fbX3r8 zMLiIRN8GI{2ZJBP0W^rGG&y@=5F&6{I|r?qz1aU%D%;%{%xxD5_p`>UB)maG(oyuU zC=pP92D37Tdz}Gpy#gExXDf6T&;p}HY@>o77zQNCqXn#`O1<_F<#V4U|9fv|j3UVK z-!_zm=c7o~W=E6fPTD5=cJI9bjI#J{?XuDI@N@k8MM`akq0FonN8^`)@BMB+s84{^ z#bwX|G~Ep8z(Jlm<#=o?}Y5|;224&2kEmQlw<@3bOGp3b5vc@Jb=IOd-%Gtp7*6=utyl+!kz7) z&I*c9|JBc4(ClwTkODLpn#r(>mL1;#d_p$$G8%?L@(mWCK1HS3aHjeDG`Oz;{0n1- zX+PYg#y1|Lt$qfnjXVL=Z8I5XVq4c6ZV;?>Ip<%|cR=X|R`wFg%9XA4J0O6+Dv#bA zG8{0~&*Jw4l2#_H1KGTgefnAr@CseQxY8QC$gO}#Oaf5C#c7<~d{(Zc?Y4tXfON$$ z0n~hsT9FMvN4<&OA=ey;IAjDpmPT~G#5O)l*yH2+e&BJxT>gm|~QgKML)Wcyni{ z1WPyDt9R**JL{LnWotDOv$+<@UNbq&PcC4mpan_@iK8Hl=YZ4OE0IT$$UI`u#=Hed zhNFFL&re%?+K<3gv~&0S+lw6ZP>)s|G$V0(y~vzJUDlJ(8BUt~75e3%7uUzWF>C1V z?bqo;UQ4%)V9Po^!N9cYXrR{&umSPVnE%r20xERjObPZ5?3i|!S~nUDt%pdTB{k}B ziP}VM1&YHZhmFalsGP@%3#$kz;*dy7wqv;@H0T`|L+yfK{NvJoH^1NbQODE_1OaA< zXgIYV9VMub%eq~GP8hK!APIqsfS6cN85duw+B+-a1b{{%sOUOHgqH;)W`_4Gxg2{< z=dwS#C*S9k9X^hXA8>U32fZy6WnRU{EQJBES9CRBl{Mv8A$o^+g1sb#zg@bC81IV! z(wy|&$$}<f z5vEBv;M*Crn{i-V4t}C9ctfir56j09WB53elCTvz{hdv2|4>+gfHfY(pW`7G>#&X2 zdSy~hPE!sY@O$OPp<754@^GTS${|R1<))U4Y4Z6)(mx|2M{Yu21<`fnQHVnuz{TMT zpJwQEGN@PL1lmLaV1sz;30bOuWk9qO7YupLJ^_}LO0KU^wy+eMP&Q8w6Cq#r0zHMj zhZ3O{9|xxD3$%UGDzw%Mg4>LQj5Age#rCB-neBGlA9le&G4ctuu2q+08rXeh z3iji`c#UCZSJjNd6bNqD3keooAuJ!fJ8iW2`)* z1b7p-Zcy+QY9{;yym#w_!rin&0Ha@F`ASS@9AN~k7Zg3kb7fW036i&*M@tZcL&OPw zDVni#&P|0ZR{*z?bxbNG*31J#$or++zSk$^%cP$Gh&Xsq4h4Jv?ph^y@qhzg-#nGS-3pk!8exs>gT=@WF#)-T zX6aT;rty#khbXwoMC%-+(}sSixyhx;yYF!FW#9LY*T=Zqj!L>ao~pc~!4eh}3+`rx zOVawghKcekKceOk^=j&xuO`^WVDmy%<=WTX?C`uJ9QMzdefb@3_@`j4PTu{pV)jVb_>Aj<+!pAA#eY9_3M~@t$?R<-L%uKt}?lL(N{XYMsRsNfA98ISB89<`f@1O zzV#Rzq3N$M{d5Kohgv4GPSPw414fp(H39l6SvjvOwW_keVL7=}AIK$>Q9sn!kY4vF zcDVTkX6LTV)>DjWjx}1AZ-MriO{E*EC}*+IyhqYeNCfpjT^E^rS)w zyQevup6;$&4rhWdl^JH#rd0n0G&M5hDur(rSvW3_=&O!2Tw_cAym*3U%Lj7j3g34? zk%o?BYk8UUE)d4wr{<(RpSbltz*%_3-2O>iFNXFE zw3t@*O-I!s-KJoHmDvR%J9@sQGN6S}C?4+-rs-f?#*ZlesCPT+xH(Kz&_dy0t$11ns0O8e|8ksA~L{ zH(twsfn*tD%u!kouq|&`C2vNW#)x}Z-{*M5HTao-*|=V5wz)PKdbkQBx*zqCnsdxm zM-OL~hpB8TWKy`>Ixz<;_Oejc33MOMgF{dZZ0Py~`x@mIoFQ|5$IMxX=v5$+W}2yik#R|*B0kH@3`+z#zPQ>DS7m`u~H{7h%^`)2=ne+ ztTZiJ(+sehn=8!jFQ8}4Ck4|{WN(*!ID2c74mEZweBuG2w&Wh`)GHElCe zB13t+lg}Z<3!DpAbiVUfZoP47jjeKpZWg*_DJWxvH1gik3at{fYK%$j zexvoZ&?xpwE2p8NK50LYag)CV$lR6ug08~+X_4=1)$cxrePDzvmu+x)c5XQw5B>CK zLwdk6>tz>|#gqDi%ez)^C7+zIne{&mVE4(07~}+kUsz8CuBq?aVqPp~S<<^Q@ftab zgGN6%d6s&eT7IuDGWoKLAvv4RXV-x#>o6^K_j@_$SaV+4f-@l@LE77$r2>>04mpVO zIP5^yjGTgQ;6ah>3Uov3E3Z`TJ=ANYFmwTibeuhVb-JC+!x@~EgTsx%bf>#nzl?%2 zlqQ)7{G58hd7q2Eygy*h&qL>4Z{YxDvK8rXRnsmMANP*Yi;c4rEm(SJzjn5+8wE8! z$IJ+zeFK+y=xo$|=7{%)qsbuf5Ln^~b>&!+`q_4l7;6D^5MYax2=CCIEe@q~sA9U3 z?+KJiJW^SCs9%1irg)f&WX^pmaNnVpav*^bSKIeSE?H97quqUa*%~_Fy&_uFwfn*E zBWKkB*@1W0zuoBWRGeryr?Tn=Y4scEFjpUYP;_o0IB>|%Xx{|OvGMkwK=&k_#aBD; zs)e0`rX2o3$(ii*056BB)4};w>GMMzTq+}8fN8z~SQPuwlFF?KeqwEU^ShZJ4SpBV zg2N^xJP+JZr*c0Rj7MFdbsVYX2mi^pXkWK%Y9A4Rx6>-^g^umV^>v|bk9Y2Gzx$>- z5}9>TKlk-sKZXypZ}uM-%g^pNXZdgz!uQ3)9p_hK+AIRkhviQ_@EKy1bf1f(2t5wU z5j=}f*==rcPq>xy4JwK$1*{K>wCZ-&ceYI%rr5V`&i4zyqoeVtDGlm-Q zu#4wco0Pr&*Hq{rQ$dq&7||75#nxE~7-KVu=vDppDnB5aqL8}cyH=1wL$a-E^8u*j zZU8CP_0i;HPauStpHHMEdqj7ShEj-w>$r7ue5@ppqq|c1I@0qkl!8EYAtl`mrH4*t zS*Vc01??I~0}1=*9pLeqE=iFOWVzlyYj_hXr282pgyJfomX$^ro-7FjhM~!kj~~D@ zqGsgKAYn2^;4pGhH%_lEVrVoSQUj6*ANAz(<|jVfIq+HK8T13eSu|W+EkMeCUv}To zgq=Muw9BD;pHOQ^wHx?aBCS{ygFz$ArkhR^uz?I(DXR}r*K0%zfT!`)zzj>=k43X5a0sGWsYdt`nSaY<`70G!A3F) zlVqYr+>n9Su#6m2FY+&Launs?eJ-dRp-jyu7Y;o%fK^QDwU;PO3e6EXKwpC&T{cQT zq5%9V90)v$3i9ZSYB0ltLS2W62>5aFG{iX8K)wHiie3@#y1!9}=ja~ZGXssS8_KZT za6Py};W*D{pT-JC6G&&!b@vc{0oO+hPl{qI1<99Bf|;-eSd>B-kJb?R#fF{(*^Ch; zc(CmvGFo(#)N7Lr{<6jYGCx+Oo;kRFdKl~($j}Jg;VnT^l6@BCw62I5?;Gdxu2i?;M|^USB2%%mR3KY(Ci=#0#t?;7PnceL4vTrHEqwzB{r-Fa#XR5=Kt10hym454Z^k4VYok#F6wA5NAda zyLX|K8a!Hr7)&o!k7dqj4@h`VLNQ0^SMVEP0JR2#B@C2k@gN3i!84=8!K?|pbkd7# zS&Ar1ZFhTL2i#|0bKol8JjuYJvQ%)T!!b4}3AkjdfQ9a9c0910Bp(HGzwe;@q9sL# zsfX6e$&c}%P zqEnSyH(Ws%G7cQH-s2F52UAV8BsutQem2FC830vdKd{BTa|x%$HGH|!8YM}b8}Y-Z zfVXW}ZT?<#174Y+YkX18-cVhYT!CCx9JJEcLNqJG{d^8_B;v|Dz!Q)FiGhk~lbleb z)d=vn*4yAyXM$j^HJtJyh}H3X2Z*+gV?A)u?Dtq*?V3qY7K*a-gRHw*SSl(!-kj}j z&V4rq!P#YjGm#kEJurNT# z7O-VUaS}zH`?qVv)vy`6^9UFX*xHH3s(~5HiQn7P?|OolGoXXxRtL`V)b?9-f-Pd{ z%|6GTy4_}PMEq`u$){T4Y1ga;8cedXmEOq31D`0wgU4d_)H&i7Mi)-5?Bm^bT7|p@wW{av{TeBHH<9GzX|O5iLXk(E7$B zxiqg*7rw>MQKqh_X9}ffUJEh;J5%H~5z4w1y0i8+y)e1IM(YBsxJdx}BY_Tp^PJ^=yND#)neaLXuYCy#A~^hsbT9Z9fsg z$is+|Ts$E3Y61u7_;`lh6`Ab~i~R0~#dbR&TX0mJKJhc}H1Kf=AjOnsIGy_kISHXA z+$LK3OvMDF%($DUO&)(*dv$jU%DTggSFdjL_I;*dC$iLMP&+qD!1c#jNP852bvJSa zim0LzriX3vP4PS#+`5K)JOQ^zN~>=I*`%5V!s|+`;hA^rnv-1GTCOa!8(zK75y00k z1LD6`du@S~Xc8EU$T=sD6RJU!HF$IUFez4t3J3{oUzef$OlWZ;lrk&raEnTB>z+{1TfdR*QWP8HRxG`cQ=;je_IvxuV0uNuW=!UR9Tz zw@ijjKVmpL6GI*^;0|%fvDx-?JCpvFq&-+&Fv`vxK8<%|XaAiHMT}=?@n=W}%DevR z@0y~wlRjROR0%}4NGew$(>^}N_i)TRw#clG)SSzPq?@Q>kFsy={gO}>y*cvLwM$Fs ziR5CLpgy}N6!^lMVfd9o{Syw5NhVLeuhyhSDijAVSs+2QG?CQqQS+nh=^1FR!5v;v zkYRo-)bK)@5Lo#9PBY9vy6|A;<={?GSgD>wgc|N2`hn)@!`qOSqJ@X&)A)*e(`=KJ z$fQB8I}+SP4MMDD(3r}cClHB*@5BUCp zPE7z0pgZZ3TjbVAEWMs3^Cu@??2THMLLzwj#TWfRONa2s6!(15vzAFt!Vyz@%hIXx27|7z0VXWBppI;dNSBGJ}xSa@TVpfgKELt z5w1QtX7j&wy-$=lx(HaRC#+S>tlG_Amm4Q~}AT^0@iofM^_KWl1&%flR9Updpo2 z;oe8?w|wA$dO+)z3}yF`iFCdm8l0#d_|M1GdM;HoUa;dhV)x=FvQ3U4q9|v$jr}*F z%#x@87q8^=>8Oq6U1ni?-HZJmI8#Kzx~dmrvzk>nlPsZO6)P*0awq-No;`fWRL;xk zXcZMYx3nuUITZ%#a+o*9a4S-1+DooLB8Ma#oh*poWD-&&LO#EKayKecZ~+LTp}OnT z{fnT;bEuZZN8^jmOP7dxw%>?HSP+Pb%=nn*w>f?nDgr#KnHk=_{f#jM1b+W^|7IxT zXS}!5zyZ@*;`I3DXO2SKndsWMY1+vzUGs3ToHmrP0kt0T+PHT%6T!He$J29nixYG` z8!J#M-2fx;a{q!{6)||obcY-E5;mX~J@BO#B>i#vT9C(h4o*Wh{ZSlx$yN-G#XD@7 zdq_Rhz?V{5Nv8wZ#aE-|h{zt?KIRu-1uTmuazw$c6NHPV80{NiZu7tgTVbyxIR1Kl zz}-bcSrqk69NDrkZBaO>4n|g2@za6HehdqSbEV*ubvR|d3UVkxOtkC%tK>b*!&`3&^k~$~ z;i%yI@5E=nhkSnsc^qk>m7nNot4@p4y*E@O_kf-*9YYwS)0W}H3_)n@$DF@pa&di|RjJ$Q@;aN<01 z^5Qv&_211WezRXVGLL9labAZvIFy6vap143aN(DanOo06(R>-k32TS!q@yFmZ0FykSRu`*Axi5uBYmlGWXn70$;4~jg-)#V-r3rGv&A_^4 z4T-Kf)(gx2{yi}H<<5-I0sZ0M3n84yOpG6-fHE8hB*)5IcpPqZfecayf|X5z4Lnrl z4wUoaIM)pEqM+mqNt)}A`TH?Cp2Uk`bb6? zP7yL5$qs9*V$bIo?C*_C_>COkg;Mz16L%bcT}*ZY?BMs5);xPX+vb2%!WyWCPowm& z)>rVh>=7$8>S=}|NbBLRytjTJ;19fs?YF@LwT5hv2ukDxuHJV1g-*65a-{jv3@M`~ z02WyNTAOze>9=KrJ}~BlCI};{r62)?+_YPm_2`r>_Waj??g$ndjTRlg-I+aySjem% z8GJ(RG}gd^Bgg2F`&#kXp=TwoQ}Hk;mL=wy&a7|)NRKw>n}4V=pl#t)i?>8XtSjKw zCV=Kn0$|1Z9YKUO6LK=aLco+J4O1{rId+z)0~hTK26BGw&0)ar#y;drl>Ui#`U9@L zbK$8f5a+UdD79QK^6|p)$lhhRG^)rImt125I!LFrVLv~l^E!|{l5GQm*2Ag$6}V*h z#*QBVLwW{kaMqYV;K0YRJrt`^)<@K1K%XgBmU-hVoXbw`u3vr%|C)IRN-*ms4$ zW?md4Q-!4Vop$~#k?d2;s_{?_4kLdE3z~t5y~4T~-^djsIG8n9dJ!J6nYxztbfpA{ z?#hr)9s|bA_~HH^V5%+?Ei;9vpWA*+o3$B#K&l+{y?r@M8- zvo*hMO*Z_ukky{WzPF6q~e-K#jVkDtr8|d+6{*~V!htZ>7*cNl5KYO2;KnLDnpW0J^>y>60M-z0vt_w^XK>?aMrmg!w^*@4rZ2c9008OcdNyL`m@ zOS1D((drW1$CCztL@w|rL_J7BA%)7tW2o?u_LJ%k`}o$Jjm+nXa<>3)&uh<^=fU4FjL}VNgD8NZ z>q09fvnCmwbgEpDB!K0D*Z%rhh0kBcWG|Pn0D=U6dO(8zKq>%ne#sC*1X+x0k^w~l z#Iibc=QCp111Qf<9F%m0dOOP00(L^%VxA+az44EjXgb%6@Fq8M92j#jutaj$2fN2T zVTl5T-Gun&kZ<-K;9`*ea_X+tN?J}$ zx-V>4`5s_V>+7JGf|5O-YAZ3(bEr=kAK>edXZFFlV#LXao9QG_Q+{Vt55w<8Ze}oH z#9_U)Jht=k;Q0Nw_BgBp@O}i!{=ysj z?@Ag6@KzG<_Vkc{RtlrHQU`C|6$<7FNM|REtUpY(RH-49vuwj9A!w77FXT4FVR69i z+Xkekywzj%UW$@d^FXMs5FSJAk*Sg5j0m(>pc+D4YHkN;-u{~lgW~#T280i2JS?M$ zxcCMWQ|?@jsXPVA*GtXa; zls-h-P$EZ_{y~0ZQ{}-2VDwRja;)FWe2=KVHE}$oNOIGvVTGgel-h_7I;5Ve1q)qP z84{$@eohuo0x_c>WKsg=+cbdSQP%jiCk(wX6vFf?R?DI6`)VT8UVVdXiNkWJ03Aqd z1#mo7kTm%7jcJFJ^FT{FVYsh!tWspzt#K9|C(J&l0bj^45O=nd9AZ4~STHnnv(WxC zi;o04?+UKu&=B$bqifLRChhn;QcZDi0AdSRGtQtg^l1j}N{7oM8C2G3c6#yFmuF1G zl4_6NXM{?;C{nlF&}9y)XH%>R>HhHgX<}8hsgU^6cEjR(t4C1%O^_06d~u>OT^D2r z$5narEXP+@pv;p1MShx$p27koC6G@NPDFoSQUX={tsz0etIAZ~_FfVf*Mn2g6m{X0 z!MnUEQ8>`jxv%!$xgvsLZexUPvqpNIiSYZ0k&8;#%MTX4_>yvEK*4wS)g>SuzcJTU zW2DK4-5UE??W~0g-Jy)q;G_g>Oh43BxW7!-|C(n}h5DHf6h}gwr-&ahn+!a<8|a?d zv7iUo7<(~^$0?08wc*Fb-*BC+#N5QZvMIb=a$r;VM!*J0E(km}O{r6y8iGWx@z8Is zcXUGvyo)S{YZ-2~$(3vw$^`$EXji}5KmXae&)I1hs9i%ln&Zx}_1a%4M)bl8>+k$! z#lehD)ME}|9q}G!`yPPbmIHoLy zL_4b8XQ=#RbY~ZA^zBQ8GgHxFj>|Uhm{WQ3`e^~=LVz&IJ#mQP0=$1IQSdXt z$q%ucTFML**YmHG)+R!ZvBCBV6Pd6Gmr1Dwyj54Qq|ZG5cyPa-0{jeL|CkRr3Iueq zPVA|ThCg;H5oE0&8h`GlB0I1Nr3E(RP?*^ZF#4G>M5|Gvi@^gw}4;^0>a2^x` zr@f8!X?ncGc-Cp+fs8{yF!ytQKy&0T49EX+hC0gXnK}d`J`O zNkNMQum0z{v%IZIKuMPXAh=U;0ao82<@mqhhQM0QD6?cn{MRqbKzVG1;CF@8aUi&d z@L5ezVsk1?ZiSSL4X7MNt=Vf=5>diLX*QZWPUwGNv;~J%Hf#}#i^2^$U5S6naOocb38AQzszDjiABR|ldBcV(d*>4R7`w3)Y62DJoEn`ma zmOBdpVY};uZX;;QwE;8mN+B-&*}OaA$7mA*J#%)+NC1U#zp}|+gDD27n;JYV^GL)V zpMrC7mc+os_g?^h!7RZ{VE5uxC`c{4#=0L`K_C>X5DyGfk7dE2%D>zQncM^(A_1BO z`&m1NKpX}7$WYmWy_jC)+;00ms3<&vQ=sGpxA@<4XXF7)felv!D9-!)3&MzTr0!J6 z#e}YV&Pbs%!1RUN4wP_RfsQsEh{lx4*J1Aa@nD>Wtk9@#Sguw~0 zAD{CyzJfh5g~RW6zW{JBP2y}W=te?n;h>@_9($i8{WH(H%g3XcFohxpd1$A31fZYN4zKEItun6nHP3c%*E_e!rd7>(0mJp!maMr zJdjryfY7$!>ozRr?t%&ZpDY+6-G%V)^@U>%6nO8HKnZr17@>*?nv&xv13?;{%N!w# zyDW~TkIa$!A|58|0f_3Lb+sTYm?09F0-OBy904Nv-J}9bb|OA0-+`)s{YG)WU`bTi zVQ1Cqey}Eehw#*bzFQuGW>tG`33G7%jqnp8H$J+b<0x7{7XV539aR+wkY&X=a^6eq zQ707f&o6+Awu2m7L5MF}QxW2j-?5Ac8e_Y{NVgD>WYsBMBe@S?kH(kf{dv*3fV#8Fz1Q6@Yy6KU zKurA+Ea>6%H6@C78stIw#sWHcDW=25> zKd8`Kk5pk>mf6_w$(L|+$U{XESCtKS`-o#v#~Vx{Oq{U8BUB}B1{%fL;$jFIP*cTo zNhSewxWWSoh5VP8*2huH;}23?O}sF=8xkHmZ=c4;XK)+h>r(!&c8`*Z;?wSbkMlj) z2h`cOo_C|fF0H;9_JQ`ZU$`-hNFx3oASHeR^{-zohkpNed01K5k%HE0I{k@xd$r>BdgfU7lCj2^UxID53Eu`za(O0GNMcPACKFQ z+pC+neYO%xl9H9zSB(&!heza4}U;A;vX*;=~2~`s`-dcnOgmM7;jE z&KUtK)da`9D(VWB*GgEYjt@FwyIrfpxPZ!gX@i8^`+wXSFeo9C#k8om0pywrA6IJ) zNm6+>603cEv+mZS^YX?sB(?%O?HQ^DBM}eCU%ph~IW&iNDbOI6f@F1i$r5O2Jyksy zgE~JYDFriSC~Vzl`y`Fqi7skppfqtB^@e=ylfd~}FG{T~(K~~t9eMW3gx71--^shb z*M&yRQJd%DheBn%TF7PA^wmtB;p=lT2&W@A!mSkN@!t=jdx#P`CR5$qpC2%E$a*yO zoR2eph+Nuz=JCRSnD>LTAAp__-G~`CeFJ*SSX!r3FY(?Rx_258`;09ggO6V>N6@XN z=Ss}aGv@4*Uk`_UkdH`y`vV4$ygLBUZ{95csT{g$Pxx=_aMIJ5&rrJMN}FG=`_yMq zUK81zMN^mGtF4ZSx-xn*FF#rUcR#RQu=S9Tk};^PZhtu(`ac#TUQxDE(XjRS}hRhDM~fEa&>{dj`Shhd3XgGUD#Up^>+D3NzyzT!(#8VZK780k;}xAD?WGq*~ji$B9eOdC+cgz8PRC1|Z zzwf)m+{^r+OzMH_--KXT_*TJ&67-=a_wF8NvWuR=%j!nk-^h@7WKUxLS7X?96FEqP>4D_=XdxAcBagKr@^X? z7AAuK?Syy73x#8k{P%2o2q=3iu$3$sq z8bAB#8E8Teo1V(^xnto)7WA8WK(w#_&SG%uyw5|-Qqq6tOWfiCf1kgXlx4nX*6~cG zI)fxbTj( zn~B#CRURXsmfeaUW34)}(C=Ni>3gukb>_9$C6cEYLzQN|>{_Uu?yd5$kDBXboZ#V5~ z|J&_YB1+rLx9T3U%;(=L)~d994#>eM6JWE&U}b42L+r2F4rCcsR{%?XktbD4B2LiThrEDdQhlfwT>h;~E_eH2#_S)YU4lEbF6g;x zLF?$TEl6IRWQeqqen$Hvr7YQE^u zPwp;c%J{olBLoEZC1Y>@n4~D&7Z!{o+n`df@*V0;=l!9i9lY-VeT<7lpU5!uxjg>- z1UdsPOWm^Z@Yp?sL)2|_b8BU{Hag~f1g|oq^H~SeW)nr@3wStVx1^1-=k!nX4xz4U z?EX8rh?S^PY<%McO3C+i_3dc9_tRP5tSm>btFZnL?pG!RS#-Xg$>qeEa(& zWhtx}_O7Ws^>CREZn2(|>K{s3t4)Ts+Go#)401q@1K`%Y*Y>ae^@LLIRY7sx@Fqh= zn;{96=ifwU3Tffn6*j_fjl9bxd?pl9#q>~lE@(FvaN5jBE%Gc3_R4t&@bmV$vXvIO z_ox>WiH4x^Gi>;M*AX9r=+~Rkkj;JwxY{cL?*R0a04Av&S;;x5SIHTJ?tByIS!oJ5 zcjp>d1N4WY!)?C*i!tP!tDjFI4i`*Mo{^|4Dv7u1HuW%-&RpmFvi`fQTx zP=N?9L3P?sYJiiR1Hm=9_l(kBaj5u<{8ea1k-RqxWjjDS>NLEA-o<#-x~tp+;e{)B z)Ct_$nQ=SALk4;FUlhze2K^cJv$OSk2WoZa7Ago!;h*2+a|@qz=V-r%goL<_(9{c> zVfTF-kM{d>^6U^nTE+ez(H_C6;Th}S|9j$!G5V?e<5WWEh3{40l9*5BjH?D-x@LCTPI~6GZ-dxEF zW0dY+38CY61~;>aNyfJ~?u>;ouU1-+Q{Nkot_r^$XZY=ESro^G{>kQqd4L;>Kt{Sp zHI~OJLBwf!G5j>7fO<|39usvgd}UhM4}w?5yjR{AI`!tWbXDBmyb4-2Kg$HH8hd}O z&d#q)wTjqvu=g}R;&Yho&Z8)Te0X0Bx3<(Ph|EM>ColH! zTz+o(`7!_7ni}~8h-@5UdG(5|K0gZXhbM3X{ByEihANHG`hcTSW?Cp$M6S(z;Z2b7tC+g{%H%8UDPvoH(~3f33fgzN>52z- zT%VPD{HR$`0rl5RnW@c{sci$>BKt4PXQaH>nrQh=9#|GR^c2IcHGC%R^Q&w4ZkdGJ z%!c<_&*j?TV|l5mRi5Yj!QoC~^K_f-cB=rMjT=L@t219#JUup+p0-jLLQ=!0f>Fw2 zWaVyms@&d`9_3QMor+icmgz+tr)S3@AgOR}qZhUxpXKgY7%F#wN)Sggy{NMzL;=Oh zrLI#CUpXwt3)zj${#lzJsMv0g7jc^T&DsG?7ZbnlmjPyT^K7Z-O6H4q)S%`@~Yf9JLOyys9M5G;CMxnIbu_p%o* zu2LZ4)ZYujqkJ8wOe^MA!^(=C2I$=u0f=p~uN_2YrdH2%oH8k!c8e{8uX~r5yNm|( zm==OX^SE->Z@a9=t`OdHuRmvREhP4M6oLwY=X`i!flaHIAM+~^avBn4od;ad(6##6 zj*QUtr=3|E-lp=i{Y5X@U%{6r{!l<`yR)LHr04Rpb@4*Lpm^_u7Q!C%7-mOw??W1} zzZU+`@0SA z;O#4ON7i{j_!ir=GPVK|JLSMpf!c$NQnmm(cgyqJ*%t9wGLv)}@rT!>wFrQAy$Dad zv;AkA;yQ%l+q2igiU|0RR7Lm`(9g!)I^l6P<%8IfL)Gx_V%0mcA?&UDLR*K9lisF0 zz9g~4x+Hu0fzXzQ0PStctsgQ?)L8zj$xAY26j=VTD@zhhlGuG$`OedCbDCHE!V+xX zT~*9)(Z9CA)Xf23yfWsdUrEGxPnf}OgK2Fqd=d^{z57RDB(&EQK4F1Rn8y5iUlCQ@ z`xDY*Zc6RJyyo?6)GwQwXcb!|>-01k(?x54@!6D1{By{P3$AK+kY)=)?ge8+!%8$e@@oq5PK7i$~=@|)Lk|FZj9-#h{7a5+y z>T%FFWNNIG^z0#iTDu1f*OZpjJxY}Wi7O)f_f=r5JyZ!Fc{C~qiu%TwF)pPyQB=g5 zRV?K-+1>DLDI2@?#JlWFb$rz^C3*)kQ?>l$r7s@Ww?wgYbb?bz=~SF+#HD-_ig6I(SylZof@E5)ZY_lmJmc9FCw zM=Gj~-d51m6EnSj;@fdWmlNL{6~i>Pf780Qlsw#=*s9x%jlZwxb748#+qUb|_zVB+ z@I>mIr-^6C?0IdPl+(9woaNZq-;&j3*YcWZHNo|yaI)ZOxumK6$L)UZZ)qj+30x5@ z4=xauYaN|qu=|-Y?oHeN%H!HEz2)@T?+%HJ-+c!6Vt&dj5RB9@5Xvo%A7ts6el;kA z`FUl5Y5K#UHs+_z0{8Ujpbh5d^##G{)j@BJ$DRdw)&q+X=f^^$DtA6F?riEUZGC3` zdF{L`5&qA}&WVWQ@@cJK^p?)+JiG5tde0wq|NPRUwPs1MXd~7TnNmonBA~?J{wP23 z-+!agf7Ju`cdQO+wqHhoFhApSisZ65#quEZYxGCD@W$--S3^mXzn!DoV0Y^ z|9+%lvdmV8m6>q&4>xs$ochsxMtU+Lseemv(^9ZlDn{do7T@Zb7axbe%{ z82tB$31>_V;({F?%F#*nCDMhd{(T%~D?)*K?!DdL6mU;?i6wPAmb|;nBm0!c@>Tj( zs{d|WLKPv`5*xR?`@J`^vVV9f3L=`%Ob2i#tfXn}P6Ry4vx~AM;#O5&WxLPCL)2Ay zsXCtS&=+D2lZ)vl^8ejf4{lsSrCqW6JzAIX5!N!o)GiUAB z52xFm&|sKq9{rWEr!X_2c3l_n6X;Jmt_nBSyV|(>y?R2JnUhWOB6X=0$&M|diX8X} zWQ`CYI659fF#BWo(Q;s_Rf!!;q;%+{JkNxAk^OgLW4Q5@>c!37?-?+|%mf?MPafIc Zquux0NJa5G;U4%$MM3@iOL?<^{|`Ug4(k8_ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png new file mode 100644 index 0000000000000000000000000000000000000000..77829f943fd9d81502cc6ea9c05a1acda2badcee GIT binary patch literal 112800 zcmZU)1z1$w_clxlh`A zYv#Yuyw2PF@1-Io@*^7#K7uNzwN(FmOpQFo^cZ&wwj=Qjs|@FfRnn zMMU07iHJbn+1r|!TN%T^NQTC%BB?3$VyA1yMhlw3BIE`xA;jPbN+Z+xp3~Ekpn#;q z{W1+CuwQAR(qflq6_b8WGZ6ywgc%%#A-*!!emfJ-L6f=HeA;x=Tz9tYGjzW-P{(q$ z1S@#JDM8*RHVNBQu&K2ABLkcc+okX zABh$ZKc8B#(TD^xP+>3;?YJ2-Qjt!CV2&-yB(-2*6#G);=s;3TC4w?9-~(N=UYhpV zRlO|eKha5+(|Uor>J0NmnUlbd66RYXM1f)5L>O(-JA^QZ*AE24#@)&9<`(A$DiE?l zvla!(T;T2qY#Tkwv3RM)b@USaI<4#Z{N5L$`=vkm(IKUn3{L{#Y9Sl9uAd4a zwDLq$I$s7+B4WD(7~p^VBErECO6tiW#R;*4952}}+j!g*Kr1jzJqVt9-}OF;llQp? z9^;{YZp{Fl2t6Ki`lvGyzZ8DlV|>#x7w|&eHp@yB-4)J`2pPWaScep!g5WX%hDb#G z7~_aQt&VX?OD*r=DV^A$g+;a}@XhPqXo>o{XbPt2*Mcudo`J1DdMzMbwRilU#P~q< z)z9O_;93LUZShJuK`gfn^0I@m~{;$oOg#acx>SS7dakxAfG zZ9MpF5byA2R$wo9|4a}6@+U<#h{rdrOe)ad{g<5s}r^c?H z)m}etf1hi7_lhyuO`a*POK$I&t5?SbQp=;hZ0fAqZ+ix4Gg*G)_$ip8)}iNY%SXx& z!AQ0+oK8=tn$5bvkM3L#4L-rc98I;gwRJ4ljeLN)tm`;fzEAsU(y9>53}c#FhiRAf zwIdG)&QqqaEiE4@+!wd})0rKdO)KiPpCITtsSttWGc5gc^^a*`ZOV*T1Mqp71aIMs zTe-`@Ru{$wi%oM`mEGAnqs@Tv+JuCTXlJjb{~AWUH~vk)SKa5ac?2v)nG zXS5(WL8wq{3@bSnm=x29l9`xBf;QTz$0QefHkg%+D%wF3#~#(q_nh>5i~{wiZMy?Tm>E@1SwvF_L0*O*hxa8rIMtnDU>m*!RTQ5wx$UYR|#@Vr;rb#F10vup|hR1 zon4(5oqU~Zo%E|@t1zLWV`Q^2jZ!h>=taKD(i$>zVipwIlo!$b)HH8!-*C|&(&SO2 z#D9*bjd!LcRH`pISKeV!PqPxbB$tSFTi;pFTQ6P5J{7aaxM2|sFc3S*?^LXyIit3u zzWAc_rQr)lUteE-A5q_EAKXS=pXHY;+6)!%%zH&KrLGS_d9mL}71gsVv^Q$|=;Aq) zX(wZMCFHWO-zC1YETSs9Dk4@k|Azj}N+o@lV3Kz7*<}AWhRGfaYzx_{Pzy1OcnhQn z`fnFSag%hD>bnFNv%4z0gS#9PNUUafhODG)r>xkP`xBf6C$e$5=0&)Lz2k337qZ>n zpMlYGl1J58>8<&#-yFOwl?W>|&g+`$o1&ebE1fKw{9amgke!yvC)Xf-SLj@Wo^M#x zG1WV%{_UskR{c-`_!C(Y*)*4XsO{LTaAGfEOl%(y|yG|CLQ zQrR`RFuALwC6=5drUB-G4=Gj27peHEQ7jqeI;P(9{#I?}Jr!!!cE47e^5TuNH{klk z`%OR8f3PYhE#@w6m{PXX-J_Xen^Lt7VSkZIkb1$MXr-vBI{$W_c0SCxFfBE$)uu^% zn!24jO0iv$Eyrf2c!uQw^yd#Oa zfO_=87h!2B$=L*EM*ffY-%j|7%8K%erm58AlGuMQh(3hYVMWQmPwh;0ghWIqBvk{m zK^3difh%cRpjvR{puRA(y-OSwvh2``k%e^W_*UjS^M4hwWD_*@ErYX)p59{ch zKaIIZxsgNEIF@J@F`&B?`q8v< zkTsdhb~8YIKt0uz?B}>?%^*{%5_5s6&G7=^8TFa{=7Y|@nwH|dv-j$K0%k{S9YvHk z%41%ca+R`{yP@B-e+24Dkbh8G*jJoV?eo%bRp;h~AB`3F{RW4tXDyL8PJ`bAWSA2i z>76v|^>OWzKh9okH21qGn<>9k;niTW5$lk??W>5lRh83lDjUnQU$f}5=-oYJ`)!#r zmGWJ?`m(>i#A*85eetC-zB0o%@0RqM$J1vmXczcs90|7fl~Uz9o0}_xXlw~=@tPr; z&ue&RQ_3#}Y{Qo=YIbbjEMGb`eDt`AOd>p}I;^VVL35@r71<+qYHE1zqCH1g-cYpE z?0rU*#<$_orqjE$bvJgYy-Zx)asaa?_#hM(Fd1wX+=mzaX$S2gLOo(pu2)WbFl*47 zPKPcSorhq_yK`M~Qt=>1CQ30fo&ev9&VrDWo_ooK^sFp~3!kA@+id&e0q1$c-cd03A~!yLy{=XLx?NG8rPcJ0 z{^mplEtFdOTiUzbAWQIceOb3|iyPU6_?6t!plOZ0ZBxCr>)lyeoePRf|6$Q`aii_W zmQ9RQflJ2*Ct@$qaps!-eg{ufZIll0kk5Ty!8*!?)P~YpLDlfrbiy69-KKNT`?GYM zY@Ccc>8FY7pSO7z{Q*8V50j%A6U%4P_ZG(&s~qmO)h$kL^XwqDk5Af54Z}zM{M%kL zCov}{0|lK53IYlb!nYNh#e>rcw))G5=$mNhpE(U8_f=s6UJdrZtXw$56!yZ5tlWe% z>v%uLy%O~{i6=SED8CPS?DB!&nXg3lZp>TudF zq2y7OMg^zCvx|Ak4a*=s<@XGK73h3$K+XL|zp{0yi?He$m;%Q>54aScjn$+~razZS1HIakaLxao~34 zBmL_MZr~hxnSm7Y*CURWe57h}?;s+!_Qnu)dPaIiQvT-<2!z+($b|d7sQBN-fnR*2 zW{!>@xfvK-TwLf~Sm3}Eb9NcUi^`})&QV^{P4TC#EYdsx5#8K8F&;~yf3l}TzUj_buxBhGLe~PO8SCpOo|CIcnJOAI3Dh|f>BDU5* zn~wbd&DY<}{_oAd3-U5RhyFjJ_#@}Pt^$JQf6mMB@1*fRXMEEi2FxRYxu}9N@C~do z=nJk6I8gujhMtd{t4`z>!@vl_NQnw5yTb0Lp)9C$;CE+87hj5g{`?u!csbFEWL7Lt zwEAUlLS9J>9k@bOa+Xxa*?5^$I#)DX%zSzJEMvLxscy(4^{V-5$Ya@K_!K{5?<=C3cddKV+1b($luDlg{E zSX))1c_cM2gSy(6_=fC=Z%kvVY@S-eDWh;)g<4m9$*ZucM5jS9az_!QS|qIQ=JD>0 zdX-5|WkU(1%)YCFd9&?9g~7!$bDtaXG?z2!jrBn!=a`K*96ffbFFRFAwd9vx*M_~- z{-GnkeGwUCz%?A}UWVRYlpb<*cwi>Ja7H+?v-j!mb@K&7h!QZ1e{VcuOzGyWax zWZ%aY=d?68+28->s`85u&i6R)Ko>cJgU=eou_4wTQ{hPxou8iGdmnw?06~(X?3{A8_m!S8vpt_IkvIs zU;9f_HI{^oJY;OU!4E?&>}MzrH$EqNQYSe_%B{Np`+ox``G`9{T8xSkK_r~Sll z@-(0PYq955VkF#;AaOprB#TEW`Qud!&7XN4571A?&tHwL9s1X%rCQMkl$ldv6_iHz zAJK(aPC7cct?Zhdu1#!=oi#5zE=}=R(ZheYqQvsnrf+)ovh$$YPz86SxX4q$xK7K> zM`+=2u(;1ll{NmPZl)&WICxBBJiI z&WJ|pKP@a-^V~SLWA5CU2`vaUOhs!l%*ylMc>h>9pLvmXxUy*3Lx*EcwbrcpwuToF zm`PNM{diD>@&0v}{T>#gUGSm0(yh*j8vngX%);YD2KUi#kI7j42KHEtW*zc}bovk; zgCh<5j+=<}rtjE&X(A@Oj@DPD6w^-TZLU=4%=Em+8KAm3Tpl%%MBieQE z>uQ$mEWG>WSuJ?zcFB-dvk2?7zUxpO&mo`7DJ-sfXk^NBDO;1#4A=J7$x_@{^T-=LQ_gNF8%og9x zNc~ujA-vm0$Ia6leoSqIoDPc!)0>FV8?(Q!ZU1YupC zmc4}+?#&Xf+kU)%ZKQS0moWJv2C}6a2_he-$RUqkikiJaRJL>-%a0Q2o~4koaVAZv zX{?%rtzQH21>(b#J3|&V@n3z~r%nen9gKVS5bd@IG!^OzbQL}|F{yrPRtHVjdGT>w z)w(k)ccI<+EY#7K{roJ#zb+xjzwTpkPM|U@zCV+5___QLx32F8D=(?V#{(0JyaJp2)Yv*HULJCzasV`M6Iv$sd zcm!R@o}D{9*G+kSz1N65vKetD)*SdC)Yntc(;xl_uQ6QFSN9>)}HCiDr+3QQ=xA$O_9AXg&g`7 zr$`l>r0#9KN&kuL+Pum&|4>%6>(=jHnNC~1%1RGRacSsiwE3Wo^uc6?W2*SwDBvPNrAFayWxd7t?I>Zv4;YqQKXqq zq5^X}pLC5bTM+F!Y;8#{_C8EJFx?s>oOf?0j@srti3_$wJl<6<#izc%uXC?>tEBTI%mqJLi{%~k4==>y6H4w+GzaUkd+GbghE1U*FVKIV*R zBYx-bxyt(RKFd?xeEvW}y(SBC*c7ZOb2Hdy_VA1_7zQ{XrCmKym!4%_#a?6%MQ=WR zewhSHc=5kB|AZ1IE4hfiQa+~P)*_3G2zooBs?w#7_&9**bt| zJSFrWcwi#c(+&s?N}uWJ_#Zq$4zh)#uOFdh)ZUz1i%4IJ`xgobvZB;(3-7(GlMxyf zmHz%euwe;7%P~tuZfE_NrePV@zmdEF}R?`+$?Ln!Vl7MvF+i zm~`*@cJI3}P;JH4l84MjUwoKs;$g(>Pg9xiI!(DQM@toZWrwNJPmg!D$^2gK*>k5G z{r%XQRwQODUs-1U49GFrvL>}t?Jo)Yae`g9(p-$TN3tq55;Ub}D~!9&c#}D7)2*s! zP3*^{sCs*;G6+xz;GeySK5BW=35zodo~yBCkRY%VwRbGijUwQBuj6sRwlh%_N8qwx zr(VRMt~d0ROR21G6R(XJJ>VHy#jR~8SL-h4|7Cf%;-!dg45P2_ zX;}UIroseg4veS2P-|B|An2&d$N8g}b9|LI^F&vXjqxFj2MS$`^H zmeH-zKoV0u!Px^)B!|O#j`iqUaoH(-?FyViM18(Bo~yvw0E>A6!$F{7cjU!frFo73 z=dj7g(I8-@?0V2{^w}i>A@pC2B!6|>mr_rhF~fbAhA+aRyhQz{IDPZA0bi;6PF~{f zs6eN6)5X;3MxySn`68jqf+AH0li?@X6jmvPZc>|?MQ4_-?VMfC3s$PJ%Qv%L?Ou1f^Z7uOI#O@bI09foaPYO`RxFSNcLd zqK@-cEz?@OY2K!xbid{4G1Od>BE->Mmdg;2z&t)gMj1umuzrue?(udnPq)R}ey_Bq z51nT}y4LnIXG!4kTEzQi1BZ_;+}!4Ag0a@~@n-Av?x z5Bj9Ko+m-+CP2GQ>7*@}bCwqqJbi^q1(BOcjpuMT&3A`9eD+I~=1Q}s8D6q62I7oy zEg8h#*IB$LomfG&uPzANug(fGykiw}#CI+cb2`v@WWT5_=0=oj>F-Vy*-t1+P;dso z@0%^uo=`5&2WH`>uQWJrX9Z*T<%ycCHmGcW>Ie#>uY4{WT%R2Dw_?bp_6wDN)s9KX zKQ+d8y%KN=>}|^*Ffo%^B1l{-zVHa+ksZMp(|h}jrDftJm95)LO%3S@h^t!}J~}!+ z4^Fe*cL$Stl3acvyc=JX9C2$ggr*A=h`j=j10ltlHD%ZsYCIw4qs?A-l!qd+LSMAp zHsV7sypkS+UeCF0rCq$FML^5HR^T~cRVLX%mPOu`j;HOtzd6hEez={pMINNe@O-~J zSzO>19KyTx)uD2x>EUv|O0SJCgym!1?@##Wb1E!i;U1tO5EaL==g(@yCw$J2nS!EA zu4@q*MyroOI66td>~h#HX@x1t9<*bJuv~wpu|;5GF7A4wZ`jX1=-^iJ9Ke$$W3+qN z&WW-xd8|sx@VZQXLBy*z9gzc00;Pl2^dbb1%dOzHEGkuMj*7oG(=s%j$I_1@i&vh6 zF*H+0@m(gpRj&{|3wSDj1xG-N>%%H2i+>X3+|D-jV37PQ5b|1toa`k9Bgx1LhE?b7 ztnxC6=Q97}wIQ7WiG5qWOd@@XK=b?~JNH^R%X)sQ&Dd8+Ay|)W4#j}$CW{!M__f?Ns)MAhMZvbAXsko^*=_Z3tTQ2V~F?O$=}`LG=gJkM2!0P?=+_= z^>j>>P}2C738i9UuX9CGo7K)x?qxu(*=rN^-NMr$%B!MVO2YC5b!S67d=%JN<7r{o zbYo65KeCUCs&cxat@=;d{m(`D(He7AG5 z;AMwlzI437JD+d<0U9FPMNWyK=2n$$bzAAF$HYalj~+)&s&`?W<=w38`61vhnN!o! za1{;lA6@)>@da)F3D z0tKfN{%!Ib@Fq=L!4dx5NfVP$#Okhd*C{Fj+iGD!U3|IRb)k4>6=69JiQ8uaI}1@& zA6;O1Fl+6;TW~=RjV6r0inMpMkW?Z*qGcXTP*)W^ za-Kxi8u=HnOS<_yTqG7e_KYmUv&HV^-Krtx_iK5i@Fs=-wOJwU2;r^D<5e1hnLmR@ z{G%1g98v5LvNFZDp2rAwqk;>>U!!*AkhQEu@;DMqH98$Ga4Djb#t`|VrDdm>Od4k@ zT_A-Jk(rc7VqAt3@o9u%=8Bj`H6l}EI2S4kEuN5~r*R#wg~0d~i4*a$X?)lYxCo|f zwPnMzq`w$NmL*dLqnsdmd^|@&9{M@VuQ_er8D@<}n$&@f{1zbuX~tHRWYy~CH>xN< zCxrL$dW}27hmceh#frrk*AInfqSBc}9|P9E(xP*wLqqc`zF*Os>!&9@s=$GMhL)1? z*N0}ro)RF4XL+QO-1MGh>O-wvKV6M0pU35dIfPV^%8C{C;CYtKR2ZILsMEJ=GK6Ki z^*TVAV^+5l9e21Ohab1$sfK9ZgZ=N!1S*fl6$DD_)_8KnDf!r9%d9U^xsdhPVvBmG5Oi;t&r9`Jv?qiDmPKr-+FG;dU6z0PP^U6OKKOyh zkx4Al{Jkt{s&7@mw=--CJlQY&2H%_j_~l-&5|Oe)xFZdEDQ!qUaGq0lN&Mpw!}1xp zd|o5;ecJ?`Gipel0V7g#pZ&w3C?RuqkYSqwJN+pG0Za%TH zz>3WP{3N0e1~SuIx>TRd)vh0V$fCA)cC&TH1ByP#KD_ynRET8xqD?oHozOJDC(cE` zI7Xfen4*hAZ;I86Q&^wWmsMc|{V3(lpnjtAN01EC3Rc1^Jq-d&DyEA_ho52hO;=0f zGWd)ek<-*PqxpLtm$Rm9swDw|i%H*pSw@1Q9Ij#SCaHfu>V4Kmk$g}j>&X_)p|&fD zN_x)$Yc4xu+wy?WoQ=L;9o$5X^XMnwyktmA==S9!=0$J}KgZ1=Sm`9mU!s-yg{)mF z2U(%OM}%bs3J7XxDeH+^h3}5Tx@xu>0}j(hz60yyN+E)0*3qk}Wd9ZcxvinJ3p@iQ z_5|&fW&t_H>0Wbp0d2*@{fhc`-byx#62+}rHA``)sjo`9scFg9r2a(XAj!Vqq~28O z*zs8e%BiO!Mf|4b(F`JgECU3>pLn4YmUut1y09mx!@O0q@ovWVs2w&)VecqN_d zc6@%-sr~5*@% zN6z)C=dufWf5$WZWKq6|>KU*;_eWgC!e<4Q%iK4f4UIQ^{DCVj=VUP&6hUmlf-Ell z<3s}Z3HHMYFcj^!+85-q_@-=&?7!Vkjm>@`*pqa8+<~zt$MBl2bCWDYIvhDAZ3Dzm zI3C_(tfXh~^SLOiD1kWdsN!0H9PjJVuEN&{q)P%QF`;Ojl!JibDUd)TSqI7TikmQL zgr;@9&gzVG`4SA4u<~TyVZ;n|aVGLQA9Y6n<@gn1q#A+Bvyk!MLuT4uI%m>fNE7p& zhAr-w6XvwOCh6Fw>RbaHhT>(8P;Qcv5gQP9k0txdCP1OMzr=0F#;*rKq)&9D*(geM zk0a$il)qX5e<#~6od;jt5Sy3{f1 zV`o;e+(`MCPNT;4i$#_V*VEqxrLj0~2s;XXTn*X;@tL^LkFi=#$sX_W!kUIx&YQdDl%SvR^7GE)d71cMBlM2+bJ%?~!qKW!7FWWtTzFn2QM==caK2l<0u zxt3z=GLs@-J-}rFj;O}Afi%vBp(6;^9K!rA0jxqv+!r9|CkL8NviN02(8;!69kC*G zte40Yn30@+QA4!rr}n9wu^$XE(5y6zk$g7#nZ~A39O-K-t_)lx-=!KKT@+Y2{7{JX zw-c``8@Q7#kf_h^!Sx5f&J6;S>uyoOF6aJ2H}MP54XF6_Xkcb1a{#4W%C@;#m5^8% zdm{;q(mlu*6ncz*@LBsB^r;Zu2c$=immq3VE2soU46*KYwCF8`HmiLOxR4wRllhSw@QR}sSa5`NKtSG zu<&5E5blLwLXdOPY9QJ%?1kYVi6~iT%@4$@<2FZ*sg2)W$)F~+;gC^!WY^p-q<&1u zyC34<>7D>EzidPYe+Z4NpP?L`cnjp0BBi((8fsmZUv z3wZMw-MZ&BVzUCPXyL7a)!ZS!5D?8OWPLljibfW$MjYlG;FV;<;g4b72KL8X22X@m z2T8JiN2Zn5Yp1NF@eD*3*C~My>ghzL&L27<43x`XT;XUI0YzPRz!=n0`=OA0y6g&J z6n$F;zLy}Il5KpNBi<5F7_vP~&Y|fb03Tu6y#63|>!g#c_+jrQ7ov&yi{vCIP$N}n zMaFb&=tP=sjTUgxpXS;>N$!NuvR?jHJ&Ncp2-~x`h~5O$fY5{s7-511nmkkyq~f#++~4> zfXdwJAudyJ{eYXIMBFc*^9(ZG8TfmeLwEr>yr;U1ATA7^D93#c19ZqAwX$p08R zCBiEW95BU&e!Oh8LY5w~>{H;s9Z&YCu2sH@9Z#*~Bu%<|bkNjNyaL;z0&x`ymnX#~ zw~$m%tJc>NaOBZkuw*a7VUBg@hZIGm*5djZr#Ve9?+qx^91Lp&N<%xqC;DjK0>Bu& zQ|`J5I|kqnLLz6Ln5_4rWcX_~Sj-3b99T~tbBf3cB@EvnBUeQoNjM zH{hOIxZ0AU@7GGL{FMCL-__@99sMAuQVo~E1Lah*0oHqdj?32`T2X(+AM2itSD_?D zA+(ES)(#0wdbN~GZt(h?3TacI0d@4p)VT6 z1f2KYI0b%}o%d2wu%BB$mDPC~&7itj$XNCno={yF4Po`EU$0%a%w@C7mnV{f%>N0m zmZ@(^b)7yP({b!_i7*e6$v9~0K5ev*xbf}aB>inNe_%>40YzwcH}ZiDAAv;@TM8on>*@L zTxa~C8z=K*VK+&c&SmhM9gv3eBU#BtErnxYBpL-_BE!HU!-wRqV|4o6T#LP2&3+lq z$jqNYc3+t$(d*J8*~$gy`kM05;h1syF{kAPGB#Ncpq-4Y>`c~&z66ksC09PWIuLzg zXRJ;!r1kk+ud3WgGw$;O9&1h#P7xZDc9V??J-y8Qpm-YB94$x!2NGBFw8jb%XZ;<5 z(JA}|eyJfi3LO5+)&}H&8tzLLj}Xrnqh6NBCX2Xq+1lK=Ng2j zaMy0-h^W{`a%$L75D z!w;AtxUe|dX0MXy9S)cb3C&z+7~vOrStAkPgn$Y`gMws|keAT;bNalHm-!C#IWz{8 zw9jiWHsctFK!ALV`x_=gT@v!^?SGOYDV5(CWvi*EIhM%{ zIf(90hjW4`yqC()LUdEd4KaHHVKTtUBL*6Pp~8WtW4d*jq;y&`Qa!bNpOFCO2vP}W z4e?vSTwx)twHE1@goONQtbU*h2`s5)ZLkHNPAFI=k0yfUCmUPqR0g&X;y+!6unT`g zEC8NLKx8JHVX_55jaJ1sHPD9o?&$Eo0hX}`_>caPnb!59Qlf@ z?=Z5dgNjr#sonVVv&T9|I^{jXvWV$$$*4M z2OMJ&zM=pU4v)l@f+YMIG`)7S&NtP?w+Wz}6`-E|>Qx}jO?$RA%uq3Mc`)Cf#ALXh z8z&zNjd=6lA9s+BKil~Q4a25ysmswS<%eH(yT`?3K?k^<8jT(;DF0`*4irt6yxju^ zXoRzq6--3C_ABB|#k-K&hKkwq>fhyDXCHgI`}QEB~bb1*Ixi%J^E?^Pz1Xw_BF8%~v|pFUhI_n&kVGicg2 zRfQAse5jr=Kvl2(n8gcZh3xMRYWh*AzCv6Uof5G#DT$fCtNjpee!R6zvkfM6j<(-; z^sqgcC{l8|33<9&_SpcUP|@5@yFFzMhqU6#q<&l(?A9qnG?U4(kHO#%Gg0@i-hB9C z+k7K_2cvt;7a&*eb$@;GzQNgg)~=fnivqG7CGc3-zhu*JfO*2{chsNAkZ4`A_yLUrd^DK8AkOeiDNP-}o9qCFg>LhE@|H-#WT_Xee zFEUSYq?uKA>p8uw>;eRnm=E7B!nq?f0+J$cpb3HBT)xSbv!AMQNlfHc~Au)wsAhjoQ{ zBRwpvkLH)m*Cv2|3_$_bK#165D)K;A|KnbuK|B3>n~Jj@8;5!z?!;Uqh8g+2@$|Dx zFGb?EC9)(N91xh7p@So7pFqwk$ZzLkTiuEv;2Pt(+yCh_2anDv*gc%aqsrs755-)9vz~|Uy1sPE*MWDO8EV&g-7ppQTl%7o3QO_fgJ&H#W9Tde( z2oui)>mg8%O?}s04qyN^i0hH+M3}f+&sSd%xo?wTWNSni?A8OJLgJ%Q3F3?JBxSNf zA_QjtP!Vr7;hi+s)zBq>MAXJ zq`6Vc3vYXe$QtM@5lny#;rTWxLiSDL{b_$a^E`5X%G@QeqSzJWSoT;!zVlW;Bd}$3 zy>Id&m6Wzngf?n!+mMJSPAt$&NZoJGceu;kFLu*YbmTkng$kU%?{ShbPENRzqKQ8z z8M`O*tEJyP19IiNjpBO31VH+iSuq_*1eIAx^Mh^op&AIw?$`s^H`7i+tfa5~il*Es z$9{{JB2DfFZrt$-KK2hs#6G{yL)U5fv|W? z zWtdlK+M%1!<>hCIroyAseun82aXrc?>2GZ5OD3t-n{7x{MSb{827xt>J7dDeC)xjJ z%RvDuRMX361{}VDYUpTiQgT^a;7Wj+f4Rj{V;xadvHH*aW@m5`Fr7_9mVx@GKWvwq z=T2l1O}H%?$6qCR9M%<7D>mU&z9ZA)CcbpupHb|_pnX&4 z@Z3Gi-|e?3U}>nV21Mb0Q2Ybux)R)B2u-(Q7RBwrNw1v(Y6B(TH#A{A(?@tI+%Kjl zTu#csj;Y^daM~@azon#?Zr?{UC;k@gIGAOKQ>5Eaj#`-3&-`>Soli6B(SVNT{ulLI z5of^O#}G!o4MJEI{sU05^AJ%_*5K$ZyrKImtR7jm#3zutU1N~Va>^{A92+ZD^&Spu z567-lr$eb{yPMcYtGhVyJJl+cY*OZ0xyV9`r6<5JNLNSw+pD@W$W1gp1u~iu4Y&wXD-Qh# zB(L{e8}pHO>=c@U$CQq8&LFH%WpCR)OrgO&o@EtZDr-ECakfW__Ohc64Ve zP&*q!_Uw+%msaJYrwMZs$?pw^+)X>(A@%|+2h>kn|5|_YXiC^Uyn(pytCSh6H;Bx* zz@lIe3u_HT+$|uw*mHqAV^WD!eV**1t7F&o3We;1mN}$J z2gD#4o~X=9gae459uyt^fxprSC6l_4wuaE8D>kr-;HOIlxDgG-ki!gAb)oYHNsxMO8gMAMgI_?zGQZe z;y4r8SvlRx!tG!I-Szq)D+NcsA`bN|06y48*W<$a+9%ehx*50i7Bn+h6oh1;Eg>7i zIdQnQKQI)kS6n$)Ex|ld*VTM{RDO=o9f?knPXdd{_z!}F7$QX7Kt=pyEr==!>4A#a zes6>w6Pf5wkI!&QjadR5mk z<_cRC2gu|PUd8;&$-;g2WcDEiz>2vjvXNjT!&0gF5VL*V`$z(znT0`%is z0CGa3(_BDiCs!eZf7I?v`uZ0@AY5I8eUgy#KX4fE0fN(Y0txiCz-hl005Zt>7bW=^ z%d$Gk-xC1ghQ#n!cnknx-+~i8EA-mn+>V;A)|Y*r;>n_T-=}cc>QMIrNOL{awmC*Y z;6d49@*Bm$TvgmNG`yG60Y~7&wc39A2Fbh2gFLI*ilDJ$(yvfqnl_ADVD{iZg_%a` zLPM=9|8q$WrTlE>i_~jF2%O7|KGF^Ve;oH`n0~hd)`SdT`lQ|iWUmixSNCVb{c;?O z*I@vXyYO%p35rB@m=v?cdWKO&65}hWEcQydfaN+~_ zNb#r)jqi4+O5f0FRE`+$17NB?++t-pR8W$DPczV|dAIK^kM3C>P&5>&I=&IKI@kq^M6jamJZro; zrj(P$-Gmo0VHd@JTR07nZH@pmIx-D(E( zf_~SFN!1IsCmb87576*n#)q>0*LI84+ms>sxddqLZBb{=&4w>-XPdnMmFEox|8-`1 zI{PJc3>B0YK|NYfP$SpKmF_quRbjs&jSHYHA}`VV6gJEAZ3=PXN#ZZ>3*-p_RVE>e zjmvBlLPV3sZ2Z~t{-oQ!4G|{tsrZA8J9pC+O#t;LSQP1})0Ju<)bi8kjn3=` zKA}4iU+<@fD-Do=2*^3xwrcW&ySpdAxk~`@jLGXkkr|$+p@Oa76y^DxhIWpHT}d6< zOht`SVj}@R>kl+@OtBme4J#ohgsQfzmyxV$Pf&UF0|cK$#%sXwNQC-4xl(bMcGYnJ z7qA~#n=}l~*V!xDDLE2RQ{UNvMH=s&xY>oH+bC5HO~j}|^vOoAPu4DmF61zoNY9~5 z)C+422td>PyF?4pD=NM~cWfutw^YCiH+t9IieT`4DZd+&VR+R@>;M?WP)-Mbb0^t| z47*>r?pqwIC;pd!gCP0z_^?qqE+2?-#Jpq4P>L;cMF z1I9jAfUkFcM^A!&g37N(9IA?~Z^{*DVE+!aWdlHRU+8KU-cL{xKJ1e0(*_HQse2q$ zC1NK)okH&nlj$;Tg??0dbHwq2uRL&94BZ4HDCBcB|xa^ z51@pyc$6XHZ5Yl(ie_TBf^c~ffJGiN?2na#eE!G4Wm3bT6)WuXw$IsDrXtvj08cD@ zG@&C0z527nOow%g*>8>$J=<@)0K+hqkh4f0jSq}|{S%8NiC-b;j}?0bvAYOJlQ`CV z1jw*cEjLYOqon2lJ}0p6ArX6IRyc0T3eqP720Eh=eiznW)XC4u}J4L0O+gx-HEU#i|OwqKFxg2tNAq-friZKS?W81Vepw&+Fg8_JGGJ z8T9XdK?UJ?U?>?|nVp+PYU_$S8r1r13}+!;E> z=-p$~8-Q&|B#Yz}{)A3Qi$v`CG5a$>TGnv}Y*rmWL&mb}r5Z0#NCXtQDnLV}lpu1; zC3+A|0W9kWXq*F=#UvccOB|~(mN6gCj~HsUnq^Ki1vr%eS-D~NnpWl8w#TLq)SL7K zSiZL$z4KL;RFYwe@06a60Zd%gsOZOwY2A8)5F;P8DvNl}(_SiOyH!!H<5qZMz&IxW z#_T5Um{$<0#NHW>yu?I?{R3KjhBe9Nt$KA&QG`-8<&3uUyD&o}L+XG4M6U#~7dhaC z%g;=dYI3Mo7=^wd=H~@(3GB@NENBGi)9$ohZT^T*a`yAn^W90AvZhOoWWZt&HSz8i z<`@AAJrT;%I_BG6sQYQPP^$Zyi+S7yXUG@kki7ARu^X1Oy}n zi38Fpf}%7i9ZE^3DBUPsf`ml~B1lV%bSNSy(jturBHZ~SWVQizu6?U9 zkZ{*~e{aVGpo>yNj=@wVMc{UenJ|tN9}oOsSm#|?sKn(inck>DlcPn;Z!}-L;8+3( z&k4Y(Kf6V5*cQX|U3-zqXQ^!Bgg$hFU|3ngSfn%qo+uYi3G4d#0oG!SSsxjNi~kyfiK{p2{*1E>@Ofa`+Wjcn2IMl`Zjpx29_ zE5DEFvfT?mf6%FK>NjHyCX%_JOH(lTaygN^Q?AYQuAH~{jXv>@VgRM3& ztKS?@4;M*skzFc%OQk!Fpoq1EEn3197&6SfC(tAl{dvmi>e7BC6W>$0*_`ik1+jo4 zHcJb>ABW|}Tc%_8szl0Fz>SJfpblgFfPFdX|Jw_YT+AaD@;T;7rA2piOSlFYy-WLN z0dBN0q*vnsalX%@yFyl4)`(1Mybg(UrLIxC@$II=6M$cxDz^0^XUqO@GK5lnYd04i z3gFP53xHggu6%2h{IsCPo#ACCaWFO8<5K-PgHGKN-t{4;JATK%Mh0%B+0oU+gvu_u ztBy^8QWex8*A7<9DOxcoeU>i@3f`O(DH1Ge>fW3``~G5uWMIKA>21FU*-EQDYffs< zs#^qBir!0>x6&GRKrSt7o#VS5yAI^u@=T>e^a;*1aj9XCm~e&UFv~wuC`26HV9~f( zsr9gKNY9isDM)*2b4=J@e`%$ulkT(mJ*?E9+FlZH*O&k7cR>$8L>lwx+?yY-(oOD$ zm#?`#ymk=oar;UC)nWjh7oG9@W3TW!w%nX?B-uim)1)C}+s(v%_{wEy#9%ikISIMN zzp=3vE|4u4wCD+Z8=mL|_P2k`uCUU7n*(iFL%7lv()jFJ|9+N1rJ>P5LhOMI8Z&V( zb&<{=AgTOl_K8%|vLlpBN?_YU3D%OqzaSz=c})~U7W^Th@}vv)#M10Lh@YXeb=Fol7jbF>0j!rf7>0OrXE>A zU5G7xb?OIa&^Yod@r>(7NfrKP!-@#PpPpWlni^Z*Oh*qOUW|7}N(5S^F%v1r>{RXZ zXm5S)uhX0D<|Vwt?A6-arHWZajIJ+rKbDGw25)qsC>X>_>3Rb)D0@e-+p0My;kIZF zg;fDM^a!FHa0!wl*PA12!8eT8-+PvUMrng(XcVVB&K`ut>yMq^&#sjF-R%l;iD}D6 z3NpLTQWDTpD)77WuO$6A2|e(B+hhYLayc6@l*dKyUC@$=g@7>$RY1pq9-m!7GW7$}Ut+a6lM8yIj4Nfd}tdI^HArG_`~+)c?XE8AZ6#Pl*DZKPy-61|D~c8 zDQ&L#K~?mBaKt4XXL)Q#;vg3Pf8(wj8D{w>V!eS?Rbr`i0oM1Ly7Wa;?XMWqnf~w9 zo$XBa{;rUV76=&8LdhXz&OXDipYhmI{GCC+964=-ry-DN;)nK%YR-9tp_Rk7fuZ;Q zi!U$@EK#_$l)_ z$|~Bz2KG35xCfzctwZb-f29KHe6a zaFv1!O*MS|@sUxZWD<-p(l?0>!3P)_LN^|Z-0CLdYMwk%LwgRz!BjbGbG4u>Hn*VR11qZbEd*ReAK!gwTQTwpY!NFM(w?0Q=VWG9Mp zW~Wdg1}J*Jv?^4;b*ZsCqYqgvvEi zg$8ou4xQ0|VWhN^f=1%>Ps8Ym=9dG9El)lDNvI6XF~w4|!;+Gb;g_h#V_@mM-(Z*>$UCeYSJBe{@T@NCS2GZF5Wgj6bu_>(?S^NfXX_yc;i zi#4np%6Zq1pEa$oHF53Kr@p=>n1F94If|%wpK)nTBI7~6qpdGV;AOs)*GkQTkmQ`0 z(|D04x8bX2dQbFJArMLwNW3P1>kY(`{EO?zSp?EFQ1P6XhTfdp>3OFjejyVXfD!cU zUzWLi`S?*lvT#!(z>Kdqtng<#K>mI;Xb0vTl)F+pt$!7dL^o>yGNb#s@+SnMk2#MI z_lep$4CEo3tQLqGd~j+5e@RJ6$Q@a3Ya|kv%yMQYG+X*pb#MjkY|I_Kll1%0otV+* zhNGujUH_vBzGoMS!PVY*Cc@%#03~sa$vBaKFcCX)8eH3LJiA(<3Z@1AcyZiQBom<$ z3ZLel&7XYuj$iXBQHN+6Z0CPrUJw`!unZ2^WcW1-4o^Y{C%s)s_Z%v=m>_OSkccI( zm0LoqPYUpWt0j_4W^FGz7X%SvCLQ_5K98p;kHRyzVYbKu0*$~mD)|1#DVS^~DflaZ z!;W+lh@doatU-bg`6Zmkj@S-IK^7ut1#KL54t{b{T*esbObOz{=eAQ7)34FMVYA@n zv!qf_k<9_(2I@Ci+giHmzQ5FE_Wa|pDKK;7e)X2k1!QmNQ2e=)tABnm%i#dg!NAy) z6+%1{h#{H>N4Af*11;g_H)9D7Tm8IN&W@}c_E#IrY8mrC_xK{P)3WnUVe+juTrn<1 zgj_Mn-z#pS9|OqNB_R||3dgthad;2>TgEZx@n4!mn^OWiuWwjtQFX||!S<>M&*IL` zjX(Q>&?xiaz)hqe0c)#0S2cxOQ^5rMN4K24>i20tO9!}BM?D0E3CqL%;lM%2(2O@f zIy7Zc#Rvuyv2j2gqGC~-qsR$EaA>9@-K(rQGw{rPg2M$YO>FqHzra~SuY+4IlPAWf z6Ti2a`LV}o!u~(zcb4PcUynZ<({;RC|G9k^TpaubLS*2F28*NpZ;GgM8l$|A4$30i z19#*<*jBZ3d?nmNzB=5i@nwy_H-`feA{euzF2M;9w+?Awig97@pNg(tkwHz zcU?LnYMz?31+LD@AuTNiC+)>}dAR=MYWZytj|`-5pVaCq2sO~vj0NJEk3M}X0}bsz zA5KSsiYGY=>z7v0{J_ES%7E;4#Y#=Ie@(`XY9R7+^d5SP2S?Mt#%@QT7%&h)U0a@9Z6OR|`p;)*8ek&}i zXOQ3z*As_NAexkj?Tmw4_|;z1Km#QM{2+4@(N}P#xABicjqmeEI|S9peFEXP;0O0` z7qawkS7mvcp{kfKz;A~pup(f<0@3n+>+#9d$zA@>K}oULdH_dXg$1tiMIl!LM2f=) z5XZh-y1=#x5J@x3>%*%dQlv!9$h>2`>kcTbmgx&vaW=3e!jF!tSHb!!nD&2yI`={U zDlpllV{qXX1Z!e=W;XbzNGkYlvLYOVkcHmvDvm7i__J+9!W^9a=k^K=w3%!{|3`+d zK9`Jea}p=4K9K*e9M2M$appnz7kTxMgddk@zDdIi0^vrFc^>MQLJmiQV1Dp0Miz4g zVO5M6kmttWkXsT$9<>Mmxk-9YvdR#Cjsjg^6%|E~oGlQ|hd-SE&$m<(xAmRY%LJ`x zNJ9{Lut|%!k0XB$UK<>%NCt4E3g#?O*4Eqt4hpmlx{N{OVY^>;dL**CvyjH~r3zIC zJ)9sT*Gdt`ueSf(eKyU9J@McgfB`A(w-G|9hsNCu4iJiE770gNXaeO&-fPbxhsW6e z-JGO(8H`o`rT5RboF?&e%=b_#LWo3p+1t~$byO^B#^ZQKx^ zo{eN6ZdXIRggC_V;R@jhV+P_wMh(ClM@l22+`$HamTI;h#_gO1**Ok?9&&!*1XUOs z{-`3tmWexf?O*?7%_DLqP8v$eD`(XXspO!6`050)*YTG}g+BnN+B6J%z}g@uf3;w9 z9;bz~5H0*%yF0F)PK2z^)h!S`g}668&$TCUhxfrh3--af_c$VAamV3wA9Ff{@Hh7t znTnYL{S5B;k#4g};^$Be#{`w)6_~`GJyr>~UIX(P>>F5-hVY1n|8e}Y<#Kp0P%O%v z9*(-hfelJE-G7GjXQnieAm|m>L6VFyV{7VBKgfn5{Dv#6A^e8^$8SLx0LccTso>1i z+@5#^c?tE8YNSh6S8*(Y4Z42h(sg#^*e!oWvFsb&Q%#vYj_i~Eq4O>wU{;li&Mh24 zelq>{N~LxipTNG$5v#3^`8!<=+lFYkTLCWj$3Ij$x7`Cs1K^Gkt6X^21~*XyV3&VE zS|FoX(DXC#XH&Jr{L zUeR?1z3ZGf3dH`~->>S zO_?SqNqA>5FlE8)+c(G$u&4!mBmvedD=S<>5y-;}yQ^k>?~O1HW(?s;93I*x@399v z1){~j+6#j|hiB#RSl=1@{^Pv3_n0>DC8iN~bN|CjTtpE@g}+&33|P!qY`X8G&6ym5HN>>yK=f2O9cL%MT`gpDA91D~xme5pNt~~k)&FTlNmw07va10(EZ}h#C#f{txz}>rvX+_hQGypNr}c;{hSs=CRV&=1*&&59Mz4^BLe}U2Mb)`GVAvUZ74YKWJi(~ z=mco#pg{yK_rwB756~(5KO2!0ux$&@dtf%Wve`tv2-O48FfbLFAarWuBg)nTBfj%m z480MLr9fmZ_k;~@g%I2d;0fpD5X1XuQufG@+lHkh5iRG}`-ek6t5s8G$Bt%19rY1z zJ@t45(jfsaz-c0gye_PPFaf;$uvO{U-4Wz*>^0=gI7id`p3)mt%oDhfys`4lNaR76 zznubxc8R@Z;9`Fox#7qjX9+LAPSgb{Hn_k7{C&11o&L-jqyknsO3xKKi4Y}r0<6A9 za%)EJdRkio_81(dEhm0~6}RWWe*${yS33`^B!q03kdG7Zt#Bv&M=rhmE446LBaRMU z>ebYvhR_S+f`4iK&H~CdfFhui%Vm;6k_Hi@GCLA1sPC8Nfo6o!VUDr{6U$-c8xW zr4jWj6(w2IE(za7?_nBf2?{Xp`mdIBni|znkPE(}9atMpNz78*Y`w7j z(~$j=W>0pPQ$XFTz0{%f4B0{NKRe%k{!wX#sc?Hgl2m_>?f=m=OS{Do3wN-9_nlL< zEH$W&Pyh?0)L<(epf6TkF70-g^pb1N7uUO;$>-fIeX2i|;rH!KD0>Z}NI_gC{Q}gUhn3D6fV;Z(n${%IHxIW>Z75h}zwhLr;G`t*eMKv3cRo&aJwk ze1VSVN7rXYw>XEb*C$>jkO=VJAKNHgr>i~q zR%z=WrSWmBz31ueeZ76!)v=^ol0T06T=tV4*!CFQ)vXk$&vyRpPxG_t_V1}l>nAIr z%q$~eSL7!fWH(mIidKjZ7wnj10@P5At^_;BGN`Q!3 z^O48-UD*-B8~dX~vwSB?^fKbJy^bqV9_c=~whD7oayI4KZKZ zP)BwhLTU_wJv^!y1tgiObE996isTGJb~%CW1m;3VTbXNBV>ZqXQ2K8LdJ@#Nzj0<0ywW$|v>fn~lD<=Pj9#2i=S3>jii^!0tK!^U z<7K);{`hz#f1Aq^*D1^@qSb#1@fq==3=v`7P_0E1J+(^Jh6-(RXI>)lXSaDycdo*Z z?1YH#iPFY=wpRGUlp6#~Nc?l2W?C*@O`r{OyFlI$SP`wZyOL)S@vdT3n35Eog{^^P z^ac4ti01tpJia^KldMZV$C;f`VQ@w-L2K^o={i2O^Sg(wTo|Sg%-dp7=3NxWQH){^ zFeY|a{fT}FKVYL{?(1$gt6?&&LqyrwU^R=slmGY@6gw8&B*`5Xn`W7xJ?2wmX9%*T zPPY0yTUZqpO;O$aq80|aO?hR-@IlIT34nl*|6O>YX#ztE06Hq|^1(b5L6Yfwz)TMB z%w^+1S>w~coQ428XM3}6 zC8z*+uj1f+nrI?|rJ3@gl0hvyTQMhTe-ThBKQ(Bn7m_(%zfg4{r59F}RlS z5Rs=Tj3et>e|bhQ+jENa#~}237!_%;3HC~U=fm}%004FxX*w}!jW{7rS$t23)_LvA zUGha=z}#t3o;jJxi)xi-bFn1;4xttaF>I*l@GPKBO*$?n2Ol}n2<1TT7aQEg*pO8b zlqea>6~^d=|J(!YQC+6T#Py>bBrm1|b?TtBI)DIwYaj6Hb54$oLVwe!t<3H`o3^GL zo!X(GaLU$HVQbn5)0bA!{JNTkK7eFdy-cc+bO3-u?~~^=m@PG!n=)klYCKv`=-UMJ zNNs2sZ^Q6F_s`Ic61e*oURFE^P`VGNpZ7*S@qz681dRJC+Q_`!Oj^UddL0l*GWxo& zCSdYh9pI_tP2ck0zETFMY#ANlqx3qtI|Ib(Z_a3=b$d=RUfaP}t#CsS zMLGC6m8hYyTjW;CK>8W;hQ^RTc}md(@~Y3{`KEgSr7{PjAk`-?PCyr!=Pkg==87N_ z(DdQL71EBJ;PI>)cTs1JHrOs+DHcr$e+><3-q8%AYSSO@d~%f5$ju`=pxOD`XvLv) z(zyYErUr4T%?XER;RngwqrW04^+Fz@X954|p4Ih$m+RpX8vUy5$YN_4ZSvLS# zv)*(5+cCfB^9_w6d1-dS3F5cEV9D9;sSLv5Lnt(8Qz+gkbu$6Zig~ZGgG*iqH5jG9 z>^`I;$+WL;^qEaLX0u1;w}WFZz?XBrr(W=Ik~eGCS_Q1!^Xq7?W>Am~??=;pw zb{+9Pr9dvziK4!9W14as8GZPJNa=+0kI6=tenq;4^DlF)0K?Kh3ZM*?W_PwU3=t(< z(R_cNJ989)McO#pp-s+D$;?8HesuA~#d`&@FkVxcDfWzO$jzYe*B6#wUn~sb{z=Rx zs4X!=Q@2KJuT!Q)ofl@OU-#<7!r=2yuZRYt%%Cx5_^n6{>1w6b`?N8B>Hq-Geg8gv z#wRhfeUmhQBTA(~qyTb#VUqux0i94r&Vzv;Qr#jFF$>QgZ^sBukPllOAV(GemNRJ-drAg?c+4_N-R9B4cQVt9orUE|{8J3}0Xqi+3$(_;fomr?Q=fpG`vB_mIxa> zZ?SmF`4={DYD_khna(-t+)mp}SdLzflF9@mK47jY_IoNYW1uIft7S_%^@< zOFzf`(0&QwKbd@jEEt`i=h{!*NcetZorF_~JDx24?w)(P>&v{)ie|fo`D@;sL#~_s z%3MpTRm&QOIY+9dDlbt~6lQrpd9e)x0M&=yQG1|LrP#*p&9xBe#XH*c2*j0b5O#P| zUV(N?)tX0|XPhN}b?vAI307m+B9H8j9Gh91PvqHGeYU3q_SQmvd;TD6N&QF?a-5I5 z1OSiWA{Rr+hMheXDdfu=TkON*h#AdFt4RpA0H)+GIT2;1l_Dr-dNJy&s0^oB)fUXB zbKs{Q{F$l||NAhf*lbF%f0Q6=zM=SsMmh%`p{8d|6`JvE(ezDhNkPLpv*jGNDj$q>Nz@ifxCHFNOp(@^_kbeJibhzEqTGy6;kZobDVH!0K>m9l zXA~P6P2pq~w$jLD$nyJn-LE%)W%y}w6?AOOzos+w)hUGWm$$ZTP7Rplm$}aMguCBM z306{}O$oaIJnk-W?OV8eICR=sKjLQ-jMw_(n;+92*>~a2(!6T z_|I`=wz{qJQkpT7Sc&JGh=J{IvXka_*%UZTO73n8O>r!~TJ7R3@!a251E{^;0V+oJ ztkR$E9;ZDKwuysZfGy=p_<-g#X^Zi2VwH9XVoEl0376SEsuAU2u{)vBdV_6%*=H%& z)WnIW<9=gH|=#4mg@useR`ePWDbWWcWjebL@v1@QLc z4DP;qO>rq;+fJ_7pGkGOZrfx5#;O)&&$^w7@Zy7Dq7-Y6)NAAM&N=P%L$^YL0ljre zggRP74*R28ajv|Ib{X(T*(sR8wIQ$Erb+*KnIq}_CFP3PrIbK-X{ zukPO0)3NjtqL&IBdqh3`DH(ubiEp z9yXh^@;)kV~t&O9;VN-XJ#f5GMW!-U6m_5skKcFKB&ox&5&imng0!H}Z=4Ho%{jSs6n z2SWz-59j2Wd+@;Fs`CZp(iYAL)%5au{ewBYBQxQ~}rr5Xb3LpAc2p z5hMVF>HP5<+Ft;oD<%Y>yI12d0Fm>leu>t}Yk)v3zVh^;Ls={Bm6)cbI5Q5qz}EuV}&2Iu||r_%?^sQ(+Ikr z$pb+3>frirSvQA}hPyeARm9)#q=oqfTg5lY7N@ee#wJgOj!4g)46v^xe66iN;#e@RSlRS(Sk@}_i`*mWuc&R~RY>h>sl zHkfjes4h^ahXjVHB;69OvmKyMSzwk!3UiR}w`lR30K#>7FFHEkugoU+RIiwoNzyG2 z@aACX_E?b)3|!m_-=#U;%_%7Rdr27a+h<@ibR?IVA%?&-uX3q9>~dwJvCu zu(Ybob;Pr6k?FWs6WAD(B7aL6J>3N+g#0uEV=G5m}rNh930 z+F*g^3=(1v_NKzjK|Z8qINW$1=`dlAMml{hla256&)h&Gok>Z?9-l=CiSx2kgxYJg zAxyB!mhsC6(3IryrMe%lKHUUwYSLU#6nBIT9wG$5TzPXRbTyfPa0@+`9U*Hk367BVY==-A1)la?ZE zDRjne@glYZ?D729FL9dQh&Yw^0A$jGm0Sx629RzW=A8@?+XQ6iHU1iz=>$!C(}~(H z5#t1;AL9SFUfh9ucIdYkklqND5c69-su>&XzUZN+R?XIK8hrH-dDjIUBKEe+i`g&Q zikvtuKK`YcvP~X6;`K9$f}5tqgkySZWm1;1FeG2xbM;G$sMb(Yb({y4lrN*3d(D|E$4(&u$x==q(aq>; zmt|+j@^^FajUxAOSu7}ApRia*wT^{2m`XsRoinqY*+-+n*RSEv&p|Tt=QA*GbyI+F z$0Htk{4I!gARs;i6Fdg&rH*@vdh-Am3l(-8j|*hli6Xgnx9=3Bu<)^img^6~P-?CX zn0QlF>4>5pv3SllB6VkRNY$Xq>G|XZ(y#Zo4J|f~P3Z4H8%!rMr{tc^B{Zi)yVEA} zZGo{@j!(sTzFtxOa3Xg5mP;Zz$zcIFPOVt3B0I&+e;PKfspY*GqG8vdQ{5!XzgK$O z=I!v9`kxrpr51k}qF?c$rN|0Ef0Z_drDj2BRWl2HDJ5PO|8rdtlUGAk+U%1{d6^l5 zPX;T3{>IgQgL3}Xq`T7(IwXGVsLRORT79&1V@`)Ab+z^B;V-*%-m7Vt1A$$1KQ)Qw z-W0i#F{G0G{xtE%Ee?NkNzbXIgSKmX%CU>6roU*84^9I$Q-GD_j-gnwP9$hy_rFgi z;G^8AebwR*xBs1T_*g@@>0YN*Cg0PWVBfguWoIE%!;KWT3nDL{t>-C#xioi}kqRRA z3#he+FttSUV1V5Jl?${_UoNvyY>J#n7qwSXA!G+$B4E778IKM8=U$KtCVxn6>tVa2 zndK1*n$=Wi~jBqYkx2LC#`?=$kd0$%~bRNV!v^?YZYt@@@0;K zcfVsk5Hvx1AlrFIIdx>J5*TJ&b(iFC`M1eE{^8?`;Dt#hrQAnPA@Egv*Ez1d>pT0r z+NGz=w^HQ=1Iwo^a1_LJ9x9{yS>Z($V~G=SSlfS2u*q0Y+1@6Cl+8Y&n}9t9#*wo9 zYGE(?7hF>TlD%~iPnUS92`_sUUM@uyMhh;8mj9)S3iH* zyT0S>@n&#hWeGaGyO*-|?q$gCufwJ65mj1_L;E8G?XTK^}rUFbV%7gCPzSXa9X9T8^f#DU)HP_lKnrXbkawyLq z#S);KDFoNX2_C~&?ZDNGyD=3Lx|tUX>VsL_X`en=(H%FDGV%izNkQV7C!!fRc3Oh zG4@?+bvgtm3Fw2ArTl5%)u1;ed!3sLgA0&b!|X?|`u+SE3K=#1gqsTkk2fJLIMA1S zapqPi@&Ss9CL+e9{I45fu^ZnW8ak;W-7>@Arm!D3nNB1SVa@(?|LR?=0b0X1cf_pV z1i@nTeF7P&M_*I!oN+*je|FPd94sV<&VFYnSC}}gQv+ko=Nk_Gl#HxO?@TAUK`Zqj z`1%)-Y%ixXbj>}6bmt{#Y3p=8B?-yPc~JKG(A#ylerJ}^;Pq`5=W0X8$_LM^JK|W} zpkZ-v%)UhP2ej^4I)DZ(kAg(#YsaQUU#dW*>6epzb`&7TYv#JvK6?*L7nk0~>`|N) zcQW#4hwk;!_mOWFgku&v3a`NFCI7uC&zXT?s zXKZI#wt^X*z-cyh)Hz=JMLikXo==G@*tyI5$r;^(zL*QkDqzPF2M3?jHBUo|%2=U! zvva|*(o~w+xrI_BVBfa}{EF#$wiQ#zx1Xw;?pHQvFIJ4C9cl;&cyN!||ChPMWl5K5 z6>qRy^N>7aGrwsI-vGHYCKwW;x!z0I0>RVJPvZVFmWP59V2FGG)ZoyMI}On zzMgtkiG^&wa@vJjne%YC!nfqI$1dseB#g@bT>6-(CB3Crbr=%QvY(;FeiPCbHNV}oIkhrW zpsBV1+FBp}GD$-mx!TJ9?EOtHFM;*DecaVO0%jS|_e8LlEW>8HU{eg6*%ct8g%!zvPwCk6~pkK_8|ZXm-JZHfUZC`q`T3r`&@fp2r2dSmHCOM&0Y+CGY|9&4)*3` z-5|p^2=?{ttE?!`<7WM+f} zT0Nw03{{qh->dhZO|_o>!~gNry1=j1U6p!EO4I8-N@c#(M;mNMjXfU2XD>oq6eZV3 zF8W@PVASKchfcA?wsmBV#`fUer^$UK&2+KH&t@p4nU@8M>_sg@ z=BSN}gcNH{Cue$+e&nbG8*J%%Xl^woPKc1&v`93C)Ah%h_LxO!^y=5ILGc@f4kuxr zhB|V(nR^#th*z%D=NcVwwu)fd*w{SMJUwbg2H`}SV*M7yr20k(wbt#y1}GJ+UsZ0{ zTL&+*@J4%#%kw!Xzi8Zf(|`@C|r3eAV!|g-|`qg@YFo+R@*| z`0oZ3Ain<)Q(Juoq5vws^*KlpC$JDitK1j1BP28=N%!2e< z@}rK{rXTYzbeZ46u!~c&)zT6If*x9D!J-?+AwvY(&fbqJQ84-uYDE&RAmlG<}Ym?}l*a^jURf5yZ&I8f)OUTEUXuUgFLZKKS zO%vYa3(~m&qc0wVaeSC2=E&fer(^K~5%14_Hc}+)S_n~(TTPu+z86mg)XG19LtqK2 zakAo&|-TVgk5pwi73>U|iB(wIQdgAEgDYVAvPuiTqx zIWWGS(=Q8|+H-S=UMj~jZ-O});D56u56%6BkO@!gfl^=1YV%PraJw%()=`{=ZK(0fzUK_s60@t4AQXc+(JLD)z zluB^@-qgkCJtI4t=I%eumHkPg7{&QzqZQSSQO&<3EJXS!2d9^1J|}|4R4woUZTR^u zj1+Q#=;AvG8~3&}M7cum;XI24)nLvtpyD)1nHTGl{W_(ClC7`@xMjJ3a&WT;s>|QQ z`qitoqddE$F8ruSOfP_(iBL3;HB2q+L58|PVPI7AF4spyrQSKnamU1o0z<`}592qk zNoujUd`XqiGoA+wTYkjk>-8L>M9oRgbcd-7*B*%;7}s*s53&#LNDupM`r3tK1yZZO z11vWC>1F+E-&(D~DEuU!kKE-=bBp>mM?>Yl9Z1mHC3av0S3_Yb=aB;$sCv-3B$qJBuAmOg}5VQeuu;Csa( zqYlI$LP%orVs843OIF03NR8v&<6=25PGynI+_VkUh|ma&4H?PJ47}otY7d{i!UA3) zPoR+Z`KTs6men6&;PSBFa&Gx9{_)y+fpE&Gk?ZmBculXzlBQqY+VetGXOsrBjJgk? zjx>;cuxDKZ86DLqyC*yy_JZ(v4msb-3zKT$qj$WkV0uEr$?IFo%^Oh&9V%}0@llk9 z2Rd>8G`*!c@&i;cMdPNvmgJLXsd9xK@FB7rb5Qm7@O)zAXs2d$lDJyE?q|VAC=4+< zv&40=o^EZGU8a%H4)B=oYogT8O3trdDt|YA#>FJP8an1BLCg57rnN@Wy&2r#AYNrn z1m7xkiB{wYT1XvqK$mFB3}tF}WpZ-R=mI4u0`DdJVu_+AnkudYy3Ng{kAsvAd*7lM zM7tIs@yF(*JZ+M}=5(5UR2QZvM8>@CG7|d^&XemaDD%!$j+(f({5i)|FBmpz6k&L> z`u*(ILnE@e8!vgByx6~AwilGhJ^3|>ZYylzPUtxIaw%l-b+bXV2ryh3vv z)nxfRNU+B(-Im`g@HIGZ`0(F!ozRZbvhV305bL0;CSEiDPmxBiMM`YrwHT zmF+&ReT`?>ZQ;$5_7N*>siT=+)Sh+ z_BwQv`hwT`O#_%LHmx~T*S;$C32x@&^63gPbh#2cqA#Rr}d&mnaqtvDxG4IJxeIyrlDRly|m;aeLt`qEw z#aduxhUr-pEm2ZiXZcYq&TMz=KG#(4(RXweJF~|kZ(b6upocc>|9p|hbYgb=;ETk% zuzDUxVZJ~lxH{12NB#&a5MVZ35B#*7$j zCexoVZY<^u-eyAJnRtY1OAp!6VpJRo9ic(Mpff$8yo901LZn_RKAnc9zPf{wj&Ogq zkq8BE@<&4+amM3hJ4Bz@M!Y-xa&l01(5u~Xo;HdLbBP`0I@=xJU~z*%cv2uCfRNbG z5&)h(`2f(-)4DW6av-Dwk?qs`W^)PxLO41%-qneEbdLKFdR@(IqSgzh3yXZ4-B|G{ zc{OI&wi6i_A28)I^6YwB?)pmJhPIi-A$X9GqJxw02SYdAWTsN6y9salS*52~j1|(0 zS^yz5{b+N;`?@ZkrzhtGb@jy=HlsjQ-jtt6UB78AeroptS7f-SFdR&ii%~>Tb7>`QI~l!lpdWs!AN$KcS&E~Fh{O;o2#p&y zoRg;qoliZ1K}k2ezZ*2YzEjUlh!9bO^gl*um@Jg*UJUB(F{m-Ii~cZc ztBJSJh#d_%ktzo?&*8*h)^C|}S^VeM$VEkoQso*sQ4dBen$46xy?b!}slm|kr7~Jw z$|l_PwrJzUWlvM_I0ok9)l7QK*?xL>)^L_L?o`FA0d6a=!Bp9jpqc3G8w_}8l+o<}mlF%PJe(|gf+S$R_lQ9To17-h_ zV_);F4h5jkO^$X{aSuEvowgB_X1dcEgft?JEtlt!_jt^VS*?4co7`z2eH_iIBSv zMp9JVGFU{Tap0@cA zxIX)t%Y)`u!}`W)rOHOGL-GRbfA1(#;;8B=IqL)h#=C;LlQkGKK6kx{>RySg$brr?9M?c;U?gaSb>x(54cgWj4?mS0!ZO37IN(V0!m=9TWc!Dk~(B8cC5BPr@S zYvXAwh%(s_JFQDf7q%#l1TCc6Xn(%0_sWe82dz4FS2M{gu~Yq5k#M-g|9mFd2l|HT z8}@~BXI@}tsS1R&r6XPiW2MlwRztrt>~#w&d;_G;lA7??)5W^IfB>;cn1zERAB&Uk zf=(l!BWsVkP!zu=5BBxXshxZmVRIW%(gcK-=ZD#>Ro_K5I}ptava3e_XV;PphKfEI zp#14JSM{jtw+ugRksLz{DPrj_ZFNLP4u;833Jw3UiNfF#J{z-yvi+?AJ8U_@*e7V7 zXcKu~-5<1iWGGAyOXKXz@j!1=`*OX{ho3)U+`T`zMdSVp7Zg>iZ}(hJr3gjp!xd{hY6ayC6yr za}H%P*jZ(^@ySf+&-|BbRR7Bhp~Nf#yVRf6*{Sjkb8y~09UqMCCs|Flo8U>+^sVL} zqmO^?sR_w4gc}@kvG(g)*s6UhBH@*~h@YV9^w=)h{w%P3_wcp3kErK)W=8z|iOZ~0 z8}I5HO;a0c=YDyDb|a!j8J=LI0wKJHCD^r=<1(NPMzSFq?_OgvPEiFm&Xx}L-2{!M zLV`VDZ3>gW%!#Oqwsv8gNhVMgU4FXri!7!0mi^J>sC%3Qe|X)tt%W@Q%M+!t2P@4y zBtVN*8He77Z4Idq?CP^&Z^q(651bR?CBj|WhsOM$mk^iQldXBiyDXm4X~M6PhbcY0 zU=u=$!dwa^*eoHVT2e{H8_A?^*ewi};#vk8w3#9YY{Rbi(}^muiH6zT`E?%G-o8+w z)I~q|a2rP6k!KT2G2yQmDzqU(iQ?}+2Bma7Mx$C8?=TQ_NZr8>!fG@?2 z>XY@^9eAjM%Yl!f(9YAzK$8Z)(1!?3lz=9TveP|8h|g0co30u#K}NT8GcPjY;}H^v zbHAB%;8zsOkAf00uKzg1I);cL$sQ$`QoSrA!VCtK2)4`cg~gulpOU!7DxZ6ggCOXv z!zkNr3T`RFdDE}jLEVISeemb*xB0FH$E0e4{+ua_z((4_63R0`m^1LRGvu{u*?7us zUX(i;p+$f)+!Q8QyNh)u#U4a_+7?rMffV64u2;;xdisfm%c7GW=+Yy3?EPV&OBB#W zD2bb!6;Bs%Mgfe{8W_k9qz(W7MA{Oc)d08QNr;zq?vQmIQ6@k!#?V~`vV6D*T&rz5 zMiq|ZTEjq0%pStERV#fiz_nw*wRYb^$3G$l4)&Wm#d>-#L0hb*DCIdE`Q;>~D7a7l zoy2q4pa1f5z0714+`VWs!Hsbrn+rgav%o8reHSOr|I2PUqeFxu)qyJG^~%$jOKu3O zWD|ajoJ5jLvz6pgc~A;Ssw3%t)Qpg};3*|HtV0i^rKhOX$i$O&h)wR$5p7uHmoAv` zoKkqpy&v9BQKO(YRwA!gD0A&4#9ookd})R$mG~|wLI)Z{2;S6j=}}JDzum@i6VV(( zB@m!#xlr&99pr+>Ha1qfv>H-8do;i=jo?&S&G&wI&(IS@Xta*}nHh`b2tzen57ub@+|6T|NIH!>(n6()-e`46X`foBG%8o!{75Y=0N)I`Q3aama`Jc+gotI)#HHyId*WZs#NCs-h*Q+a-ED zH(p$nJgg$`F*qQLQ*V;DRhMoA{mCs8OtH|F%v-D*YsdVgZOfCY+jg(4t=ZMv$9VD1 zlvpu;kbSB-Txg>Iv?MhPsQwL&nd^% z*8FtYt86hm+hAHd&fIY4cPk^iKwvYR$t+p?Rt4u7(n zOI_Z(ynE@a!06>O4W=eNg(4cslPh~aXJ_VScAgG@-`+2JII2w^JUv{Q{K-_kc6j}F z>$@Lo_En~}a_cT>>-#z-Plw-mdFd`BHyG@XA?t7C2f*E)_7zRP^W$}zMdF9N-^Wcx zxNABn6IhskF!m!Y?|O-0VE4II_vbAO0RBBrPx4HVRCUX_fg zxSqc+1*7c_a zWP6OL)(Z*){);U?4|4A@d!Su+^b^uXQr5ydTIa=>V`k%nW+ojLW%v*7=l3>PC_HnL z;nzNX_vw@(kH^u=C2B@lvNm-!d*+MB(@k5St`{p0`#0PXo>OC-OSZ^6!UMwsP5ECM z)IB->e-ZcAVOe!iydd%+B8ZfL($XOzDTstfmq>SsNJ~m10@5I$grw3aq6iX_(%mT` z-3`((>ww=kcmBBdJ~PiVGylT-p7)%+*WP=bwSJ*{>{+H#_IC5I&thsHk9*9ny|p9H ztFKYlMaa|YTlMy5(Oow$m8sR;ll8lmgwGSkS+>W+W4e7dw^TN^mKZ8ea{rvHFX>y$ z7P_h)HveMXI&c@tab}s>{YZRJ^wO#DM0Kr(kY7zuR`z_P^c*|F9Y*6w^8y&(n~vG6 zAED_zr5<9%gP?JB){fCPMgPfPk1i0McGEEBz0Msh$(M^HLp^ud}g6M@8ir& zhI5ab{^87w!y;)q8$|wC+1SAzzfIrb&Xg)^-uONqEqf9BqH~F$u%73Jo!imy3bD=W zvIGfY_T;?2#hJ3Qh0O*+8!glyg57&3{wE7V8IGRz7AamWhiUITT{JhgXm_Y`x5N@j zr5oa-PvVf>@3DcI<*hHgdgSlWWqU89e#if!%3vAV(!Ygk>n%cpwwXqNtN_n@@lIpR ziNWp$96fhTlo&9k{Y8?{c(15xPhr-em_aUgbU7`Z;NPDAaRl1kcSOYoxnj(420^tG zxh4zN+?8Pz)bF@bvJ6O<6G_|{1=v}iDHV{LK>kH5R;8Bsh3E6*cFbN>)HllX0~~K=Rb`XP+e}~ zzE$uI9XRWE3WCbuKL!6i8HW-t481-4$AGBb#UFH<~o#zehHjVrDz4z(?*UMr6hAkEU4eTj;^XFw~Z|>J3v1U{$cu12;;H8$o0g4 z{JyivOtn;r&42lF%hN6IFEkBg1+qnL*V-)R9_W3GZSk)qXmQj#{=?$6ankN3!i4TG2UrfZ0I)+e3xWHp;;nSHXk!2W2qb4;wYiSG3l6k{1q zenC~=3^4lvh4o&ihvN>lc#@p}P8Ww>)^{acpgFL3fFL=Po$s1$0jR%$+j5Y@e5Cj` z^sbyD>S`UsNO*#vgCgs7(33n;nkgSI0f?=Nqx~)rQ!Y6)ZH*O$_A*;;elxQ})rCt4 z>|_D3n(1)8P~M@(1bm52Q5#bYJ(Q<=l3G@UeNUkvM=wsLNQml!Kgj(44cS!^b*1e* zw6Km~Xd-lj9TrQ*oxR^afH9RI44+$g5#$Jc?DtT7A>q+|tHiVg4KD?Y+(NbVn!#jF z?-T(%N3Kx!_o+hy$wG49_)c^In1iL#^toK%?$K`NT^w~e{e``f)8(66n;Yz2at!T3 zp=@`l#^%HBNGCwUr(#)Zn_b^|;-Y2@N9=n{{ZfbQE$Ds;FeJO@_kcvQWQ^CuPrr7= zdrW3@qW_28(3o|FHqd!^( zfLvfJEf7VFfG%&zUYCnnZX|di@-PwOp-M(1TIff{1L6XE=1ppI7t_yIdY)4LCF~ z@E@6cJF)?RZNng0)|CiJ3MR)u`3&o+~%o>H&lbcYU=uM5 zzcS5wjNvpweYeF!IYUmV>5&&1m|ILYQR?)42M53535TPnr5C#{_vH7q64QSEFs~o$ z`S8_(_azF9H@V$zuem<%PC$p1%v1kc2Af{l;AqEi)$rDs+Zw;V`-WeQRukz`s75<& zkG;?+eKamV0^F)uBVNm5?pR+xbca6T+qG2Ssx}VHm@I$)WxZK^Ah&L=<)YP(*WZy> zaUio;vFnVhw`eaW*U-GD`Qt6ohVj0q%|00e4%xaX_O~r-BbUfq{!__ZreRyapW)Z+ z=CslF5n>NIKRd5r@-8}vqD3#Mphsl>sIk`bxxs7mXhHK+t$bv;Y{3C4%(*^a9NqwgO-2r4<^o^2?wPOBarACqSf8 z`1Sp?2O@+&Qui~9m*VZTv49U#zfN!fWiiq+UW9C%udh_OIS{@6m>Na3jGGR{!y z-Z$ox;2$%zzP~?EdD1r#hvV7)N@-x+D-oWIUw&fYQpLt z{qsfd8$}K)2K!D)^~MT-%=2%3l^D$~NN$QUUBmcNX;_6vX$382s1gS1rTYS5!ALpNukU8xnv z#w%)m*rI;|@DU)fn#UY!G#PAWQ4Q!?7xzl(13hAd^G)?NVjlim4-jw?%Nxr^XL5!$ zV&dgSPEdRn= zp}+ml%M6X1gUeftmu&t)zw-JpmWHwtcG(2sqX`_lWIjC#N7anzwoQOG(8vqKcmbrO zCEuBxrSgeS4zF!>#K?2NgXQR23-zJziZm-0uPiE>a`ySZlc^a$kS|IRFx2wY+J9NE z8vLp+l8=?GZ&7tFI=TAFCdIVtn~3SHH=#p7OI%(5H*-xQVS+c3ul$^!Uu7^mB%YKVa4+gO=7|YWou2o zNPLYj_LvYw##FkRgc?wu~J4jY-XXK!oQ{E4F=d7UsM2ghIC* z&I=hK#j-<}hYvh%m|~X!Yhc26dd0JiGvAO)dNOP0jQQ)w zMi$E2-1Xj!$&Jr^GfWN&z)3vJer#tRw7XGcT$pVZ1h^UP)kz$5ULQg1ntGMon(2s( z?JcOEzRSSHUuQP=*6-Nr@f2178lhzXsMx_WlZb7FUGjiIj*@)d?K%6z79;!SClQ%x zwW=$CnfXN1Vl>s!urKD@#zZ-1f_-Zc2qDjwTAB&!l7F>&=|!kgK3&vcosFEr&LPk= zBq$&koq9yyrFM+fESs)8M_j4cuVEY-Sqb5(K#v7e?b|H1>E|^jzcRNoZN3DpdBFh} zUO&fId=$CUyz3>O54HenYA6lBGgC+ z+EV>L0Ty91c%+SsRxq`0W4)KhCMd>L*p>2aban@J|Kq|d!CPv>c~3$HR7*QGdO zI%q1(n0$tSzViH!1V!V%mv|>qtVpEhV`2>25-Y#y^Cmr`Y5#Vkmdx5PU3*kt=)mrm zE(7E0aZ4BDjY%d#BAm5ha+Ag~3r~&OJ?u7<^$av`$(l0K@Mc?qB(2%p5ra)3J^d!L zLGF!fJi_WU-tHqm#LYO$H`b*t47h<=Q*qzd7aKw+$~~)=XhRS4BUGt7QZdW!LAJ=b z0>m8J%yj;7*bZi+)td}*(ovWv;Iw0GaY5`j7G{Me@?*n?dmBH&Y+=1~F8KMw>if_W zPT_BJq4L}@4Lv%ywvTi|b=IHO0+!;|?uNLG=7n=#s+ z!do%gyc!AWZN<+@A)s0&3dVjIX59$iEOmMCx@R+{(@y#0@CWA&sv3Y{D{p3U=XB3W zq#v8h-wStecPOicKF=Z9!ksYMSICA(Z3jnZpK8S>!&+ z?LHAUEO#X%D7fX(5@Si3oljFLdcv#Ivefqp(UO`j!umVS@gyEk#9E88jC?wl9_$BS zOujUNlaMqJY2wI2tVwuRSi+Evf0aWZLRF#E$uF=)?YPG_`@7-|dY&oeB%xNfho-b8 zf}gi;LPIw&Vkyp!on9O=vaHW z7k)8T-z_sL-q z_hY(l?bha4TdjeeCq??>S9%+4e6a~(195f&Hg3+sMLxS;z~4UV)}?Gt#T87?W0RXy zx}cX+l@TTRy(KgUgd$HJs7c?JOmoyMK6ur+$4h6a!@HQwo`QMF=XyrUnrLFgw7KJ< z4~g7Px|+POy^A}Jf#;e!SM1S6<+eA^Tt_LF)c5PZ-El9%l>>_p)$lnT^ z_QcJ|-w@Di_}Ss|M*et1;JV3EXG30_=GTGxA8l(U&G0-^w^&)7m0|pzzO9}zGjZ(W zVc(vWH|21tEKPaUGX1fW#mW<1Igl&7HCsMTw8B-(ikV{cTFG-Q4LU|Y`=rwO&>d$ONhQ{XHF(}On5~QA6eJ5rX(SI`aa&8xW^;Y?O2Tf0var6BT*G4mloVNP<%9zCJ$J_VKAFo*j^b*d=;A#Zy5fPU~ zE9rjv89do%v@V0LEdR^SGjIKEQAdZtW+QnpP9D}hpe~FkX;H{eA7*FE*BQe=4HhRJ zH54tbTbr}uQf`f52K2;|$Q51W~dg2SWh1=K48AWE8hK*STB23f@YfXGk;{M3v)?C}Zciq4GG3(?mTRcy{ zzT+>-v4+YQ8N7vR38vZ_4Ewza^Be&~F7;d=zIeH;ej< zGg6VQQ60;w+K@8omt;|GU5se+B`JS2`w;r#=3n{ZaZc)UTCe1JP4MwxT~-X;3Wyng z^KhMg^$W_nYD>ar=$CG0R5I(ruJ=I;dbgi@avacpxDxcdj>$Fu*?7cb9Xn{93!?UWx zr++7lw@!A!aG%DaJ{n7B6l!zztYp$+;We z`#cT)QZ%+K7j2VR$JtIQZ#!)5TlKcoQuG|)?bf1(6jK}0OW!hJFl?04L8GPgivWL@ z(RaQ#Z+@kZ@O}6?minfxC}yDJJ=)e!@%3w$=kT7q_Fa$QDE5^8hn#>~#)vz-wm?2* z+eM0Y>^`l;4LW7RT95eFW4eqz>5<`DKJu$-^adXl21(RslH}&3wAMtwqDR(BCH0zD zHu_1^-tLq8w3ZPcOPvYOrwEN7R$9KcLj7VVa`+al+3zSw)F1s0F(&$;u(3Wf ztJAh9*YnYA;)o_~O>qobxQ1`xAvGcvT5DvBzeFm1BqP?jEOiO>E8cm<^gF?Yi#{S3 zE>BA{q*dQqOYNwPm;L3$ar#F^Pp@;p1(ldP<4#YV8An}S+v`0-${}}KRjxp@s5@e! zLsYo2FLDJ8Ep)u^5?zM+Y!3c;*|pzbb?BId#7@5<$6F$i3aYOS5h^Z|rZrNZ>S({N z`qjD;TNN4|;{*oVOX-+N8F`oRhFZ@7u2y6c^%}mgy7VMNmHE5RU2nrZs0?JzDzIIjfA$x53n)Hh2Yf#jYvNfCc4#Z? zwu3B=&b0AVQ(&wJ)f!qi{$O#*5Q+N82vj1k1&U}17@o0Ga@%FFbhI`;=)v!-qgnP{ zhQjlB4RY;=fg(u`_W>E$5@Bx^?^N8F`RJ0dFqB1I(Kczn7&5}euA@s~0VGlJ-=77b z&T2UAU{^uaMGNvr0CRfmeLhy2cT79caliQe{aJ)#A>QR;<{kWD1ps^qF7ErHZ>}w! zy;=$pE+e{kfy2+Uey6yHWRDebfqYp}H)@P?N)|5ky#Ee=J$Yydly@C5{w@ zF1&xI6R9tI|L>~kXN&|T2P>`LAD;eLF}Vc*@CvYzi{wxT&XN{D`piOk`Wc6k>w_n1 z{9BwX;T6S9C0u}j_B{GuW=dG9ik=#Y1-}&gFOKw_51*DE-v=l~{&=o}5K#9QM}HLY z)_=G9)eXf}*0u~@(bl_w*Z=srGziqP#6bQew1oFsGOu?58t-bcRVa(Kwe%aDW7(IS zix~I+otzuAjd!}!F0zGbSh6TzUIwVX1n~P_U$KEpa2wTPxB_@J8*m*Ct0i-ffDop9 z*#iLYy=l?}T8@3w16MQgpLw$so;5LEDH9%;im?yliJeEt5~XK)F0cHFkAULu(C|M{j@#ywhsySQ<0Ck-`~Ys1KESyepb%B|I-aX z48{_%>&bmH?TAiI5=2F;GbbgJCiS3HG2)JtssY#-019rRSVXs8FF2cfLZv1X+=ELN z2A6rwdjr6vmu=TpcLd$@9;i1M{nKo+u8&yrRW{l2AG&odl5`O& zQ}90zk%a`hu71qp;AjRJE{1=bBfz5JYbc~P)%vYgfYQN?w@xq!WzyFt$7^27yQ`x! z+&kbjN*!ukWIZ}EiDX)a0DNgr+qbU=tfv0WRx~K!irEFL=57&bFGa8o6yep1Ww!T-a(9c{ZN#M zdz`4}+vOQ9=3;q4m3WuauNHVFV4nmd5>=(M?<}%_Hgc`rV6y)7t zevAGll+nhLLwN7^KL<6GCnEk7u36kK5Ssc(!#!UM~e``-~KQc*xkZ8th&ushi2UI7G1%k$_PzcRoifzSZo>x3eaeJ;Q5JpA;|j z&ewF%ieK=JV87NAsQ{{Yd)1@ul{1%iT_P0`_rAxDSuAK5ZmX^!hPDY`ti@TchOupodrJ` zl*WHMQk@(OR6g>*Nmq=_0}fwT7V(_gqGY z?`4w(PX~yf@ll57V@@$i%DGHOW$qxv2L7N0-Hv@g=XbxpDaIkb8rjlMaCS1}JN}6+ zL_aXC$1eH6@R}vDbKm@Hta8!aKOScMTZ8|6OF`Yi*1Imm&)i7k19ja*vISBBho{oM za6`Za(v4LA)Zzfb7E`OrG_k8;WC8J^UJTI zaX!V}?m!{Qul>8zbsI$IRdX{jVn&{wFW+O7qz&qxm2;a-V%iX2h*z*xhI-h>%Br#j zTg!k!1|0IaZy)~>1}VX!#3SDUyUl}#pWqiFVq?ci$_s0?u7f$tz*B7sUBgv<=)mul7@=`yExj*R(ih^lkXOo17H50 zh)=ZcMH3YlOtvYPd!n}R-JT0J1-Ltl01&-rLc8{7mllg!_VwPtvSxvWXc5O5+nQ|C z2>%U_iv1ZgT3sYN>1MbYI#s9D85v(OmtART{qZcD;Y|qc6ee};3cC)SX&6FdN}(qd zR$Uop`>TWcbC-3NIc{aWkVM}ncI(YsddyFIf-<4MIiY`}6#9PwYww%XbBtRk{zm_E zbh;kud40}=U^Uslc}CCtt9N+Yc3Quwp9%yu-n?K-C%6$h>N1?{^&yf^=+fct8hhp+ zMS+DhG6@r}C@Ull$v+KgHTCVRbpNA$ev$L6LG?Uqpz%V}Fxc0NMeHCCNv8oe8ZE5c zWFb~wnVwWRzijpO?5X}F1@4DkIf&^vkv;2;_ZuWVZ(&>Lpw+VL#QWE#9K_!s-DI=NmLqQfDPV2QNQoBo;NO%WPR%s^B3rJr z5<8@I&gYNow`MpE2~*YsDn)_!(54?9bm@K11KgMYo7rC+?jF$E7lcpu!;jk5K+&fQ z4u|;In8~`G*H})excomJP|art)ElK36w(%;So`$rdwFO<8Give zHoLrb9zX#@6G)?B??+^|7Aex8mu z?+!ibcb!kZ!8e_wa6T#gb|dcQMO{HS%ZoQ9qo^Um0td$)A!?`s&|+Bt<(BuMyu*#y z+n>I*e*kCr9s)(diPbHkg|5^~<eFL&+{C0t<5m-U_#Y zMc)I2X(<>XG9H^Tos1Ir$_(QIk+!8MOKw3V_Xn`Cr?4Fw+*CXttD|Jw37Qutex|R; zs*!YDqT*r5pI)RV|Eax#)(PQh2AV&VL`BncU7(W;VH1F!B~x}TSfz)>3NR04>K$$O zC~|f{o|x^#|1W3rmc|#6)urLWk2Dzr%c z#>dToLB$2x;&aNR>6r7JZvS3StM=7nE>r)6lk;^UMJ5BugbXz&Q&t*E4Y8RwmnB>N zKJInyr?0tPl`|Dsr5!iwi0=X#Lcd99#|W6W?f)Zbb9xmtqXe*s*&Gli$Sth`DGEvs zgBQRW{PBEk6_GaD+=uC2vxKgAJD@9OGbZc+D&gsmee3%X!TDR~sj1E$MvB)&+)YW2 zziaw)wcyq(%K4<0XAz2DE?A^><2gh7Hz4!7Fi7%7^1LxZi;@T>5kEcIt#llk2ih9l zZF6a(f(X!veE^oxRERn)=z-Dj>v{sT;TzV6EeYUVKM^bchm+1&@IB6F<-}{h4jFZlzyXp4pA+O zoN_kAIX@Ye`L|w5q8?+lV=~p?HNI7tM>mlS)2^tHe}1do!wmbl{)Vzr3lZ^Dr~s2}jLXU4S!RFg~0?ySMRt^}~$9rbXI)IZzy=gj6V?ODgk`aPTjJCM?&z`zc^Q zeAkG16zM?fC^yYU^6|cQSHg)`;SEEq%@HZiyNJ9-!~O?KJs%eY1!1WHr~_?DxvzKO zoc>sYP7ERhDgs`c6mhw>Gv)enzi^JZKPMVQ@xdvQ*2^s z+rKm50Z5-8#Ms%~&aLdoJEOG`oLziHpQ3S&Tg*GAf9N97QQZr{IYoJz?#4cTYvkt( zj7M|kR-$MdMCll}{aDvLN+7?ubTw}6?NG6;u@*Fi3tfC4bWS%#nL#Hzd+%G64!ZvO zRQcr}-#M0B3C{z-&e5)}Q)6EL%0n!j$N%oCxfQbi%Mt`oFhwMe3tsTBiouN1Jmd_ss_l9#HqUicWw8?06mNO|M&-&1vY$a`3% z%KHCus>b~9Pu0xbefCkWi)@=r^$S6aqEz~4H5I1+M>Q4u?Y)b=e{lYPsi`as1P8l+ zt`!%myQ7;L80n=shlmw`aM_!F9s9PYnBq)8PyzIs`8)gtHtngPYmX$bd`?UXjBi=0 z{WoPo(#VtzKU@PbX)~aSBb!1`hw!PK4>~sW#^G-sKYGu+gbc-G1rR2!G99OgwRJ+|=>v2u7jWyb@$o7GUh$#xyFekYK?B|)4t6&mzWpz$DskWZ1u zYxHfPl`ak!vaEbF|A^Z%Zm%gIVlSbb08Y{fng|Bc@> zeyCMr^qc~g&NVA`*BhePvp@)4fOo^_Cw2|w&(mxw_Fus+NL&hHcIgpi05o_C^?_}G zZ*@aihd#6E`LG#-D{Pz#FfExNPtz$uYDL?&fSl@}W#iN(3|{5EnTnLd<{xvX1v+6D ze+cJpz|z96qC7c1gDMoLIPqD*qt#m700&w3w%P02LVvT(@NYTa)$X5>AF2^-DxCJW z6r4&O7JlNa_OA>_tZvaJBS(JI;zM;`G@+w)u`{KkS2&cb+KZJZJM3}RAjQv_c)iU5 ziZjAzg$+r}UvPj2{-+}dXtk&!e7v2cglr@q5}4IzJz+o?=t+BTmP@t1ghXn#Sklg`c9 zPhMO%mX0JU)_BRJ@m}|*A9p%S~{pK!0Fyd2hs$Kzq`|A0l~;{*T}ZK z2U_2Akl;o7*A6Rcvc;^8UKN<9@^6`1fv0%>ecaP+4$1@eQSMOqqLb*m_ktmg?mXXr zqL_w33LsEi`k+m^*9?iT(Z314yDi+oKK{0GNzhvsO)8A}GV>R*N2e#p-WpTtJy~wXoMQP zFDGBdM}8Q57=-0A;&F5U#sNa&Pv#zHNop65Ie@Ru zC`61yL65Ty1hM9$YZO6CLt~SKJ^yd+Zka6KRdDry7+dnqqr`tJ-CMn5Q@FR&& zzC=uy-nkdw4p4*MwgzJK4U8F(gqjWbSd)k#xH6Q8F9m8-HaW&4il7P6a|ZwyG?dga zSq;#pEt&Z({r&Ihv+|AUUBBq*hlzkkIe+txegid}+eyfcF}yQ+OQK>B*NoT!I4t%` zzZ3Sn#;5kYHV>tws@Cdy0{mT_g1Cq<^03ZIBrSXTV^v*;lE9>sL_4V2{|?ZmDA_N* z!W(WW$d71P%>^9lm(s*0yw0rWmj``N69I||cfeGQa<0=J{(ZYRxuNQY_1>$H>%rw} z4Csdx@M%%<+1|7&5b|td*h|cQ-R=7#_8^0ZRqG+7F@-~QeUk;C=JIw#&I7=_oAir_ zy7ufuqI=CBD9;g$OC{RWkAZR}ei1vfH2Y@*hObxOs6lQSdF&kFfoe8-iOrhxEZZnU zvJH!l^k1ugYa1C|*gP}_NkqjM@4~1b(Nk3%c0r0KZM2ecbdmCQp~9rO<79juD5rj- zm5@=kzL^sqbe`-h?c8n7mIJO^T&R`gS*s*KfAj0gGRrf;NhirwsTo8Ut&uTY2MT)G z7c-3X=aKhe+Ov$9(Tg9rQ@7iF3mm99eTE8mrCj$GodS2&w0`5`TqCTPc#&DRUsS)J zNx%QReE4fZyz*fWfyTz~H3n7DK zHGC z6Axt{QNS5j4Z(h)+Dc^M;r~AT4##wL@q?4>2GG}PyCcjAJcJh@@V7T-&Y+O+ROjAV z8S&MBGF^U_ERjAhWDUC0q-!h7jPn6(=!(1yMSW`g2jQw`6TCP{AG`Qe1m&DAsJO6m z@1CpGChGZA{ROmy`WEA4LAw1TtH{t~o0+8UT z6*p3bq3DBH&)7#ROSS<`Q*!f>JNx?aIsLq32&Ml50gyNe%p0Sw{YT=Kz8L7bOwR#b zjCc3koa(4twA`VMRMluAe*X?4rJxQ(yRttC$1DF5NS7OeF`}ryfCLjHFYn6M{gbx> z=oJ07SOcLclBB}*5C@rTS_QbpJ9aA9U!2nfqw)^ID1Lkvd%+xmhBkRK{&Hst_qUzO@UoXtZ9~jr6o|x|s&gc_8k3WwR1q9HL!e!2d zN<^q!;K8QjQYE~Qfk*P7*Iqr(uA%);ud!9jQklx|xj|g$d%|^>kbh9)8}!8D;a%VZ z70Pk&OX}u;$n!^KB=yS|sq#fZ>-QzugrOKjILTt{)aa0ZO7{@;xLQn5QzlR>z|Cef zQ1C#Er^&)#v3d3C#rzkirES#|pl*iZ%>!ixc%FS=CAU039U64~+njBbNC{xN2lU+S z!PF;bbv%*N?wRo6rN2!1Ny70S0*~Q=+zW0ZR5#&sz9Ok4@HtTDIx+byi#bkp26Xph zxQqg^Ta^K^i>v?{SZqw*l7$fn@%zZ+z(}hIUPA)kaARVL178)B7$5Fe;{iO8lBpJc z&>3Gfs>Dh5I(0}0kVK!P(R3o^!df73;!B2Me*8}WG^YUDUzF+^a4#8kq^_V+8Q@)* zPZm!!lV&{QsDxJ@-F020+@qtEZ4cIYb-}yMr09JYJUAXqq&Nzm_yd_UD*cnR-1~-9 z2C&R5P_Q*&EqsE}|Mr0tixR4!v(q<%vuEDHN?13Q3__v$!t_(J53Z4o4~sL1@~y>x z6wZNyOFy8!3v&d`0$BRfK;dMcBY$eR^_PxLh2=t>cI~o=fwFIS4ovbRpkvDt3NsI! zGp$`lOvbx<_!13zm6Vr^cG4)Q@h#S3sqk3TNbI6e2L-%=$on*qE=%mMy#W<6$>HnQ&`}FvM*HvM zxWETD;mnJf8x#9Qi151?zImn+eVn4Eym&dh38II-gNz48@v#q#BI)r9wNK}=1hDKp z#JSe5pqWD;=O*5iK_mtkUTeD{f?XC3HXH0!+p~FYj~0F17@O=ub=CBbOcnu7D=Cguxrd4k*a+hinf} zV02!=9K8DdnpGLC0NuVmfv(`m)~h&kMiS*)Fg!*(d>>$pHAGQ>0C+LU#OJ>yXGtZ( zVhMv>n9hKQP_i^&gZNVWTF4qU2wZfoRXx#t37ATW!Pl2zd{JS1u>`vEajCgtC3naa zI>z(;Oq_8Qk}yOJwtBUnvcMJIz>O$%KhRVQSnmbify?}arRYcZYTFjkmf?uC-NkI?qlR7X=PNn5 z2O?PPnw^2selYvonyqFxWPPJJVasSHf3b)xLK;|v*!P$}pUG-SaD`Px9BF5Y zFV_6pzA+qdnV(fkzcGpKil}j^Se#LV9Bg1PVf<*qTAxp^+2*8|{`EYZVN4>RMgI zowXKSmllx4B?_x0&{~8BX5J?H3m!=vPL*D^uq*TFnB=A z0nl4!pU&uQ!%`~gCc zQy!jkVU_9e6s-G8iM!b4BWsF&z5yxUkRc8A6k&nEh-82vP2xA-Wnm*Nb*90-G9D+u*e8Mm6{fq4_DcQQ#K}Zj9Eha zzajnS_6NBjxK*a^tV=MY7BHlmOLzqh$c;{xHPJ$R(f_y@3c{#TFCe}yG-gp>^KQH{ zdtuH~EN?6jX01zdQVN;1y1<{vtRX*<+RiMcj2wGt->AxPpS*&9bB*vnC4nIR1V8X~ zQheevUMESbcqMm;OMFExXuGf*5fF6-uUB}YJN1Z zl~%IaK23iG2M3uQGl0kbICa zi7BKIbvp~v_W1v4kgh&&=i5He1tp?~+Yxsj!P7|De&B!`C4+5rChHq!w+aLpdG##} zcHcNd`1h6PSKk5M7MBHr_>;4e^L;p+dLG|4y}u&Pa}iD*Rw+puIrzQ++@j-a1Igz9 zw$MznyE#R$koxnlir<0J^oF58O}+oq7s+O>xB3cP>hq;R@y2ez%7y@4z6tqICc9N2 zEvHY(nx=ja=`kP&{+B6hn1RF)$%s>~=_1^2Dne+Yv#5tE1}au)8N|5BE^`~#gj_P{^>OjH8b~_{eK!h zW}hzp*jX{rD=%sLaL?MqWdC?FHyRWsR@s|^KMwS+5wmYScQM;M++p{hd+wlY#iep6EHAi zg^b*v_jozSw{0VSs^TX6!!>V#GX2zhJLSITeDY%Ove1)ugOg5;8?BF?x>KtSr-^=Y zc(^P%M)=I!ur7UXW_K#2jO9S&-qzGiL);cI|C%xpsoebd=5tA+4jb#Z@x$)Cw#FZ2 z`sJsG8=F(DCfE3FlC~AkoCnGo!WQgzNz0Dvsb%($BP3&+`A+DY{MAFKPAbI6_m01+ zUo>W}3~Y{I@(?~kGaRXNT+g7;6+=xmKxla)G`Ogpze9UVPBF>L?#WxwxGX2;$IBrU{F zuNBUOiymYrkyHkWdu zq*u0t>@r=k4O~vlccL?QD?UfZ{F?Y5b_LO-6Los=iaSyj)o7i3rl;?+U< zBz`I==GP>6oy~p@j_DYna&g?Kck$fYu_4>Yp!-55n?MTVuW#~mvAmkeVGT$4-ntub zH$2x^gjV6(dX>8%-lF3*=*%BZnPl4srbM7OL{IOCE}Ny5wO)0WEWt#yiI z8FTFRHH*u2Byy6UCLUSbl8NyiIxAXDqq&Wm+Uwa6KNtQF&xFD;qghH*iPZRvHm`{_4PpIgZ6@`n*D~MthVF2!g8QlDkh95D$X#kuc z7JO)AL=G(JQ)1D6a(?mqZ2V|R^{KU7GMY>vT#uT=yF`igFk2^kPoC&n@#P+d#h;X& zDSPE#>N4xeq*Dqo&`cOUNU&zPcAbdlnrMP6c$mmTC+czhq51pc6s_)DSDD?#t8};I z)OD{I>&VAZamMow>a&@7eBGGjgIf!^tWvA+pk$aZn|MY4jdrvNjrVU1_!F9X(8!}r z!q^7AlTi0TuhWr77#we~?_kIn>$I@Q_^WwkcHY9l&x;bNy*xM=7@#gq+>xPI;AQ(% zGaBr_X$*`RUN(**gCQ6H^$!*EYxd$yNgXpSmo9@JV{-;iKYVbBo$87SqWAShL#tfl-kO#B?{)7E^+Gg)V#SYf`d=x z-AI7ZQbj}}C$#ecMC&X#Jn#ePz1^1|yyn4>Gq4`~E%lbUPJbo}=T6>>?WMQddY2vt zwNTmcWQe0q;59ZV_^LM_*|%rv=^aWO#t2)~Q+XHT!=KSKB}sMXfBD>a{C-oKMg7@W z=C0h&E{hjEHB=|H3*#79xX5AP$}pzQ875U(B6OC-4~dJSOQbriK)i3DghO9s%w|F#@f0m1xtM#4w_%&aOpz_)_Qv8qfEUGLDgalWlUXXG zz_T#;H5Kvl=!TYI3D~G`)|>TaMaaeT%G1lmQi12gTWEHOf^hnekO^o>+JJwfbp4WF zDF+~cU;OhJL{3=v z5-xvnTG84+Sjw9LNq`7wXFRRQ)h>#Ly7)UPlZq}1kp7JXyv@_YA00x6Z!Z6Xb7Y;; z`VcpuEVYeix6MfkijqIfxagCDNz45ze>+zKVaQ=Yu56rQ}Qr!KmBr8B6NI z9*JSWOZulI3ir5(N->BfLatwE$r1*%kW)Lqo?R22Kp1|k<`ihKJoHyaRrZJe>#va` zxr#9x9h{m%^FEoFB_Lo+HB|s|YS;^LSn&iP#~Ta5Ky+`Nl?U81@or3Zx$2Kr#$R0c zXM23Ci6J=w6=_`eoGco$*PQ|$Z{S3xGi@(3*{zl|ISD7ar5C^Pw}`7I;0ww&~{3XWYNf#}xPEdJtwSFb4*`H)Kw;7Q>-4W;Bc%yi}086O8@o7kU60DbhR;ZpW&5h6+O|?_WBFD2V$R0`%Fm;KMQjJZBcy@Yovl}1 z8g>f&juJ!J4R|so3^v}<&A4I2M(FIT|cHS?;K%_jxzm09$R z8UAft8nOy2T{fqmyek}2Gmr9Ue+eIj-uD%$x^Ww$62y4)4y*O$^=;#`z*%Vc6flfk zo^h6S0DRaPe@|lmyz}m>%08gLEEbo$?e@=EfAFF>0;FO)0H@coCK|V8tu@zo6PvOP zbrabh5G|upXw0?6&mlgWhcvo_MZlMcZlatKA2ZKUiks%@>mtRyPwo}f$R5QtNjC2J zS?~wKf3diV`{v7GeRj6^s*Omc=W$4p&sFq-afrtGg9(&B*H7kixo2|K^OF9!0ZZV~ zPbY9mKTMSrkol~#oE*xwO&Dv)qsN9$WWt4tS2MaVkXO|Tmhv?q3lnvKQvkLP?a*iX z7)}W=JI=lX01hM`>?|!U?M;n44(RIsGeJ)61x9u{O0S*)u`un>__fdjy`-wzFW0+^ zfu=ydt@8W&eXb~BOpgZg=XbQNOVYL5LC28)y24f|>-WL?`jr4|*(rMGlg{pTkv`=6 zjnWd}b(D4O+zz2{#t{v(oA7ym^XOsqdxq{^lnKKdUd9>fTv53k z->=>aGyk08miNw$XuQ69mJAd>m}SaIEi2clC6Mc`S29T zA$RFkK=48Sxgg@^!gOMo+<9VR?Je)6nZ#k0cT{h$RwlMa`RKb&o1hG-w`N4~Q1dl^ zi`hVZ`Pk>{l0m{og^hIS^Bg0%hQ;U+E&$;b00?Z7kbg2>?0lh5*9*!kGhh}oO$M}o z>tYA*K+3B;Hw0&VxI?&J5iDd&kGI2Cnc=3+wJ&y^<`x?5tlIko3Cj~;THkJ3T*C0r z|BwIxO23LHwF;p7Lf*P^KNi-uP`YV1t&yuu^&*ISI3ey-y+|UmV1TCpU7J$zK=T`Bkd3;%jwaSl3dhNBGr{Fi`sn+w(D(%!=^V znqdTk<8k%oKkqY$T<48qC$_=OrGr(CUVDNtc3!jA{lOMr_$9?c6sb(_B#9&WDT=!B zJ1702zkh(S(x`juy%5gyU)&*os&04(ewmL@#$qNfq-r9JUC-0~@qJa=1ArUk+n<3% zmLy<

    st${vfL^@+yVhL1)hdI7OlS8O?o$Wm3-eZI&)k3)OH=0sTD zJZpFuqwguLSlU{$pIbfBszX-G)Qs(YWp%cQHm2CnPt>K2A^ZQ}>^q>b?%)3-LNpzXf`S?jLu;q`3w)dOO~yEwPj?0ZQtwBn@wKb$N2dAE(ftCP?y)xiG{Z zdZv@6bW>h#ZjOCwgn-=VyrdEH1O1c-=SdTO#TBLnQ4bp~n9JQITgs@8L~Wi8#8DTg zxwbzhGuYWAK^o+?}BF1H{&G?Ifbn2VWt}SM=6LpWYc>P4! zq(o*|Y|UA&7@>0wes}oZV3Mw8hHlmp=alt-QjisBG=IsAwj})vLrw$0PP-{`vy5>fMBsC{C~#dT6enrNf2u$jb~Cv?^^nlNQy%hzW| z=#tqa33uY3udzbG$Z~(uHW@$UZIavVHs^lfx=c16n+$Z4ltZ(F7OBHA-2)iU*Xu4P*2ogQ7FI95{C2`wSS@`=)QqGbn+<)D;c6YdnE>fes_xA^TPbhmcwUwW zcePKZ97Fp)&+{<5an<&(yQS%EQbh;nIhKA?aEryU%CJ7f!3Ut3Z z&t*V&+gvp6h$b);9aGwRbbs9~!6cU*P=Y(uo?)2mnf z3MR>8_Ox0DmRPw-FU|IctL#hrYeD8FEaC7f=Pxasw`wlUG>#6wB=3Jc#itR!?(s1r zVsWoW(`4FOkCMk|k#CiIQf4%O@I9}@DI!}xr_Ijk36rLVUV;{8!u@48Y!At~^P)C= zOU5Rc{T6(dB;g5>KI)WV-_a~}5(%iz?~P})BJ*GPD1UpmSRQ)F+=4=dQAp0pG}B+f ziYK*BoWT09SuHjPZ*ddXa{7Css3loqb$3Ijyvf~csr66IYtEKvvabEsbu6~Gzux<5 zDI`N^Tso}d^Ew>WBKScQCaF1-c5Z9#@*?Cf7yW&z*ko0qaw7|wAl22*7LDd$Z^^UKtoYoDr;O8)@IaNg+~N5%c-cO+~vyX`kk#_b0^KIGC**wb|H&hX*z>-ks( zEcRDLG@&Hco8Cw`B+UmspnBROz*)0 zRK4&H5qn(rTP*gX)~gD<{Z-$M2_#~1eMMr761Y)a;?0|;Q+ILI%*s%HbRJ~JaXP#fha7A5w+Iwb`3w?)eEv~S>`7Zf>vDu&0 z%@0ng)L8+jU93x}-UlH9uj)T7bnd%W{9@Iit$SETbw+kMk;q@5UonZ~E9z6#!@X}e zov9dXJCoF170yxn;0@!7g%zBcD_rLtGE;~ubEIQ7)*p*}qePi%;oK#*pPKG^uG{MG z)yk)(_2P#w)t%O_Vqe+UdVhw3RYj6}t)yEhM+#NUXo4_}3m18prP!StA( z4m3FxugGPljq_jI%*V<73H-phPdMiL!I-}Wb2-3z zkpC(#(d|t4odrR2d~K8DTHY|5Y4rH8&3CiI0KNfIW)w;6H{(8G8$7@A?2@_TQ^Y$+ zW@XmR_9lNrbQ3k6Oq%$1yXwP%`k$;+lGINJe}4Y`dN_NozDZQl;+^nj0Gjp@l}t@y z0<}xvwo3BRse;Q_BoTC`inol-m}+QqR0UV9Kh&x*Zb)dl!z3TeJJHKNx1o$2>-_YJ z$%>G0#So46CTcd99m9&sCzu`0<>^CDK+QIsLMOcofiaaz=Yrbb`ucivzSN|BUJ?4e z%te$iRY+s1;y#1BH^Dk-B1a%;@GNaZyN2*qvRWD?TAkr5Z{4XuQ`|buNubKb5d}I< zRRq^{`~96#%Vc`QG^2J6S3erf+6HEg$Xj;TjS@3k8CCV!kpE4qz)f)6?5HkJjUZ*4 zMM4=oAOAFlD9P=!pQcA=W&950N((NW=iQr-x%wl0jAlb=Bh5FswriQT>w%llYtjg!`Tq!b(S(d-J(P}%;pqMeVl%Q{R< zQeFKe_3vt`HeIwb>5YH*%?@vEt>FgO{u>iPsqVGR49P1U&75&7A>V>SQ02Y;!K8^s z4zmSZ1EVTDI5a7Yqnk@XwPT$l%w{C0ht!3_YdGOjUPXtXXG7x=F6x(xwlCWbl=dfi z03+liH7OH=7It{?!}uzuijZo&j!MhWT&!z$>?z2W;(k8Cj`$$Fwvh`~S5#7wcQ?BfF8_p?$y z!M#bqlfK{ZbhGm?tn4v~^K=#s@pqE&(K;LT!=(4GEG^)dw(93?#JN&F@+Hy}^-eD} zblp&FZn=B0iO@=V(OpPPk!peuPtCCBV@a}e*+(*?0V}txevA+7bC0OTUj0OQF^8>w zT)}^1upV(O$(c8U{JBN05P8w7&wO=y{eCXc>fN)&KgreK&;6I6F+{tAl?{z89C=0m-ap);enL z<>>i4dRt%YYDeq`ni#Fu-{tS6X2XBKaA)Tt@l)og8wi|;J-FVGmxJ}$yR4LU6{? z?O^qTcjK0db!(rA_$@j~Dsf)MQhuf8X=^e4jt6iRC#uOhwXEL%rtx$N0j8URZ{`iB zbh$0Vpo@;=m8K8*E>kubuncIhR2q@i`s=fh1zkCTHX8?iwu?pXSI!cWusg^HHW4tZ zN-ssSTT4cB)1zs>i1o-5o|k`9`;bRpdr3}v;rz~@@F{j54pQ6>jc>EpjD5b>Y{*^M zbL-FMC&V}GaPVSreaZ-Ivewutjr^xSZ7OHR0`AL>NuJ25 zXCIq9P~DmrYiIB^4r}}x1J~)>_a$LHvZ{F>lX0+~{3KwGx@P{^+(p0ni~-)<$2;D! zy(*Mh)G54sOeE}LW1J|>_jb<+yxl&iZE+@gr`De*z$Ayc=B+BnXmtBfVyzN@)094C zi+)Fm@>R1XchYteVJ?RYA+q-Hy|`q@E6C)A|4NC4dc#=IQ-k%s@g zu&!;Cf#AzY?Lp}#KWy&LDl7u803cBUU0=@!gV4n!vQSj<`r9Vqbra2&hdCZQ%YDEm zodbl%$H{En^kjz{-y1(|&DiF_4N+sp{m&9%VWkK(Vp{5a)ebcySVXjK+0&e#l^j0B z#=i$T8tV->^h!T!T&AE$KfXOK^uwQ1XZbRc5G4)R{M+(%}dLF7sGK@3oR7!L-B?P**Ux&kbFPFK-ku zA;18&9iTYnZfkc7AS=63YINso$sJArAO`BvXMIv7Jl_1GmK{MFDyi)OyD7Kmy7>Lk zu&w#P%bTJ!0n-Zuh3s*cZ#p)~Eb12Os>6V?=Lb=g?N2sfAlrS65P?-?SBU@n=a}|2 z#Fkh%762!@2WX{g8?yvmPZJ{v`{@+oEI8!!XS4`a|4sw+P^i=;PF#xqihRi)=+F5ZGPqLu8;FLA)SY2OK~$!9@Y3>r%p#k&NDxK%jTE z?wvm~EyIF)eB`6wT3y!19r|z<^WJ7Xa;R*a8L=%)y3!Z@$}fBarQ{o4e-B)%0k+Mt z9(egI5|A5!8#1my6Xoa7AVllqiZr9#7y}B?3H}a{&kM*z4Nif`w#pwyBgnm6gr$5$ zpv}+fYf%ynWYRTyj?mb{2h+ll36wC58m$2R_IA*~98Zxu`=j}87KqHMJuqqroj z`dAP#6@KHuLz_OKdyT@Xqt5lM9!c+Lxz0Cea#`;NH?$4&vP8UVIusbNUxI|t;UkUdBaBT`?Y|V*t zG00x6AO?vStiJ!WZrG;P-w>H3_en56Na^XZ2g<479%G{)b_t1#vO)5v`riW!fc;Y7 zYED4tWTF3~+5c3t0~&-^UMG+yLgJEvm;Isl_7DiwxGIgnpuZ9eX!IR%n>Bw7J=xCi z^OC$pe_xGa2CI7DG_+>{b+=W(+%@gIr+<~{4kvu zk&fN-O=bY(e{2nK*C)9iYMV-JU)!tov`5RMajpj%#`6wfNMFpeY%eh3JEwp^0e zoe4x014>I-COVI}avMezJ0CPl#?K9(^B2J(4}F3%b_P8Gw7;m4d$7OsbH&K7T`e~! zduu~uj@Ih_sQg}B$Tp9Z5y2hd6GP33!VhHgXgBrgFuH7n?PGI>!oIg*JPE1gfL4!q z9k1wBW1$u97Nb}Y8;XK07Se{WuQ6Z|dK+tv@FJaT68!2=uO8MW<0qHTiIw#GYaTne zvwtHB-0<%Iz1$sj2}_ni<58oYN%~=A+xqX*f6mAQX`tN}NXw3>khL~%Kmpbco0Q%<(x;VZHn!g^nxJ~sG z$r9KWS$O=9c`$~gc*1vS0*^um)dc&fD^TpTAtpN86+WjIc52D^4kN4$r_nihqrA&c;t+mxiOc5&{OMtZgnKz$L#Tb-cb29tQa$N;EoIZG+tQ}uF zbSpzYv@&uzyl{3ZFJU2Z#XOD@M^=R5GqAGd{yJ z1A$Jkbn2oXWab;6+Sca}JYzI-&*Vx$9Os5Le$QkhEA|U|s_BU-e6Sp<#m6zL75~?C zT4g)ECd`D}b5!jC^+mTTCc$P7tnAzMxran%Jv4nEf#YWR+DLh+5}2BFwqY#aOEmbL`C|3(5WbJp4Y~hc?mk%}rZoRte4`Y*681!9-ZG}V- zkt`6^up?=CI0*&YUWt58y^=3`^iXnGcA;xdIXOrx@48Ahk8FCOC+5R)^7oh@ym>oH&jm$>89sBiV-vg{fMr_hyY}fGf-dtoCFczLW}S2Nz6U&k@r0zb-O2> z*35z(lAN+$|8c$ZDSrEXZG&(}_u^y5I@nR5D{*CWx+f!pY!@0~9U%~Sdrr!!9J-Dy z;cZ=!S<;r>sUkxjO#y`6^8-J_yymSD>LnS8nJr=*cy5z)hlQ?%DTAtTGQQPlo??Qo8e#;=cmy*MLno23-}3Ch#;15|O0v_= zS?c_yU1Km3XN~pPu6^9msNfM<%8(HI)Twc70Yz~5J#MdCW8>!E9PWm$Ihr|5&6~m6gQ+eP2j2(jRaYc@AqKKJUz{#8a*r`EyvoO6I^VbkaPlM2O6SV6 z|M~v6D8+#Z(KI%t;VB#WMH~X)>-%L{R zP$Su77meE4PMa%j56&~KfUDLQv3q;(EtW{hy}VHS*Cs_UZ}jNURCxXso?+I088GKiC~@Ims&lp^?A z<)4I!Ak~vnReikY;E&Ip+#jvCmilMNl+GUibWXim`<)ha zlyopaBoLq4|4&le-k-+G@`*QwlR7Fww(<)hLJOXtDN_zwG_6;MZFG7ksWPzm$l$XU zT-T*3Asm0QD#x7^$Env2JATDuxjGNYe4;t&@nhBggAC|a0N4QTkMSq;lefdIV}lgo zWoNRWDw4F$AN)H1G8t`L4djjML!fSyb3~^$`P0WDvHwulMf@%XtZAS` z!PJKJP>RAwtY$?l%7OIV?ky-S8Rb=Y9iY>W`rv_6wt0e5 z^?rE1&N&8NW-?Ysqxa1O%>Rjn=~U`=pwkov=8cp0Od^afN(fq!R`77 z06Oj=~h1IY}fN8 z;vPZ(T+IO_gD?XPzKxI3ydhnCsarfh{n2;!2*1Qt)6!HlA6axS{I6N~xhGH1WUc^I z2`OIqQdIC47! zBHBu62mRX-^Vkk{wQCAOouJY3D}OK3PJH0OJcrMGCc&p4UVoM%k~3HGNA)K@t^}Px zeoS6sQ#Zv0na1<|!y?G;il)zpI|B`}B+Lo{-Xbnyx_-GIOaweK8d?zlU2&br$GNZA zHRIfqz;$apKV4%6$Pi8k=^!_u7)~GpbJha4|A5rR>jNF9pFdEa-n<8L&kzr#hbzSB z!h?oR;VA(IuXFkZx14xZ8TkGUKLX_WxExFegp~KDFQW#?tFD?H{Mm8UEz-{-t=}!V zRq^Q~QV9A83`SwV(pQj;LT&n+)0k4rr7wJ(ghe)R{dv zXwKryH-6xY2XUPQKjlK&>DazYT1}t3r4HAKg)H1LH_|*3)30qZAX9du{-l3=ltvF?Uo8Yn@CS?9D660HRX?k5lZLR8A;y%HF_rLLPqM1qzY?sv_-UyF8E#-sffvBzK z0b2m=hyBpBg5k!PNk^Zedmpw1_cqZEjY4G9o8;5;;vh~MkzAg$@yolHiW5RxN0|O>w-}6sP}GU-m48bcbL2|$(v?(P415n2o+Rcvm?$;R?P{| zF>s#D^*vo$U%Ob|K~tHjBvm{S{NvBs;r*Qd@doE`uaU}rp%N4ULC+t=HVMfe@_Q=q z_=BVfEDMVVH)YFRb6D?~GzJvxC8Dh(lkM|V3#UUK{INizc#DeP2&xn)*VK2mj52Bp z&gg`y1z8YG^~=ZngY_tE(1qM7a6F$#^3=LPUFCp*!}MNw(;nPs8;z}f6>(FK8J2>$ zq~C3$nEw||^e|^&8?7G84P+>O8Y?J!1>@hM+NyO!g2F;L$>HEBg ziJExB=2#Yh0dG|g-D<+>m59VO3ux`Aa80z%(fuH1o%JH+3|6ra;SMG-lPs84(!}~y z3HyG}w|BK+7kC#EztqADp}bHg)*q|9R077DV}#Qe!<+WixVjsdXvO26w`kk4<^=jN zutZZfGN*YTZe*|}zmK462RGT*K>imbC0;fqew4AMCi-#uAt>U_o4Sj?Fk7!RFkj9; zpJ*p49(48O(hRxkd{ogJB@yvs$vthVMMS}Lmy{4K!AKJR4zBkWO0b#Mk1OD_>99%F z7Y!;=BDqWZ4EM%5vlS}a-koNA`9{F;OZ!9GPh9JldPc`EXz-&@#7>a657C*i;O*6S z_x-a|zqd^S{00AG2(&B-J|>FD@9I()ip^h<<|d*tP!kW#(it|t?_Ejwj3Q1SJ7Q1G zT8w!HAxnRWSF{t)!B`7&!wocLsr+lMVviBl6NF z*eS0_Rq6U7o!if>-#6G{>buzHlXQqe3^p4r?vj$P=Z114LEg{DD4xZ`G=dsm`}N@N zZoM|HVD#BRkN2h-LQ;7F4o4-0?`n5!$FhX9dfq5fZ~HbGIlY|GDZAhS8e#Ib#<9=* zFLsmJB-Vmp$cx<#mfu)kP6vGfDC{?OSA`!$sp(W?7idzl3*ASD()VtXygB=ZaJze< zY)@qSW3$)$q|{e2-^_Hj0_G*SUIms7g1^4eQ;=V9tsDmap`Wh_Lj~{zd^51kzaT9O zf!vd9{@^IQ{2BFXNay6cetfZC)skJv)-A-pYU~YQn*3I5fZ;U@`g148BL9)4XbP^h z8-zM>eV}S}FfA}5W~~zSt60JsBl^$POm$wj*mK9xqLNQi?7N69lFA>^!#V{4nOG9f zXRB)`pQ$Rb%N*At6OS&`rQAq5EkR@L)i?B;qIbXK0;O_@c;#xIDmMhWaweEgUuy-+ zJBU&!>--xr^)^HJnx2x@Unfr=Uwf%stK$l#Gj4q&HDQ{QofdXgwwesdvahwP0i#maA8g2*4Gx>ajAR7V2&UStqwcrxQi)A~iVOp{O z&l)EGOiA>eg@6jAhub(&QLuT&;c*sHDtn8 z`}}3Ou4F;Za~aV9z(JM-|2$>=#fjos-m4QIF5C_Ed|G@_C-gK1ZYKnxwsuCwHnXaj@EGuq z3<*Fg=MrV2EH9ov5^`)vakUviif54iR|K~UAO9KrTZ9BE%!jVs)&(EPf8s`+b~=Oi zB(E6v7HwHGI3$Rai#2e~TVWZz`YH zhuQYO`gOb~c+AM-UG<0)7^zlhTrCrQ$RzD^A@A$mTgNNg=NY^Gj+WNKy>3E*&Ckzx@=}g?M}!3ce~s+sr~pW2 zkoHN!;W~5tSPVXx$V;C~#I_+}v(ZD8M=@e`>D;Bo2mR@=p`Cj>Aocs{!D; z3FbyZQenE~FE>>`%%DS7 zOf_ummMRmvA_N074AiiHiDt-GnWN@fP9nK7AucF8%J$xN^&bWZ$dVKbk zC{8Xp7gv!w4LsDRR|EPMCojB9j{(hqdq1R<|EBCPFq7ei!)Nc$Ch?x!-yFpD9HS&J zSi~89&XQ%;7codH{By0I;2v4pC%^cUOyH}}bi@<@s|2xfPpD{seW&9~}0 zL)AJa@sIvgY#$$l-v1a;XH*%BGII9)bnnp~fBQ3OSWRscwyY^~eVntMSXVC>${im} zRcN^HT*)nts-ylI!iq&9dI5_A71JA0gnjhf()-Azg3810a}&aG4cLU`E3NCtn*iFO zuv%M>r8<~}mW?D+r2|Gzb*7&jfy1NIiCA{A+df!NS>U#_s6%(&@gr{^o%&n} z`%hd9@V1zZ=6%Qifh7tDS*^(e&U-%<;N;?XoFPnvov4>iqB+?K{SyNgzsG+O4@&`_ z3l*ytbn>$y8eo=MyWOsOFc{@+&fcn{+% zEwjkw;i%eJ;4V(P5bqaJ?5(Ox%G=;r#33t6h^zU} zZDsk_dsogCs*Em^^YsfDC{mirec?+t31tVBm zMeNuX=WhF(GaVUa4*7qdDuK!2YqbNs7Jf_orQ_2XB`taUSdxAh-rR@(6dI6x;p8?# zI0Y+)C&is`dV7*7qv*B!t4!p>Aqm0%e7GAvjD&36B4=nmYu><0#g1VH-*(Ce>OSn5 z6h)ZG%Ca2Y%B&uqb(6tyo^_vkm{u{x6oF}^{8;>_#BXyuzu{62>QU+sEoQgIjgpch zh3M8T-xn;}^)w<~%Fl|^V7|#i@_3%*3Y*&Uhum9#*8Np~<6~RQ|77h!b|*6md5jaL z&s~T}Zt7v`h&<)d-PCGm77b^!@d-VkPF$ z+_%|sao>AQjB8nF@4ln>X8VfH&)56#so)>Hr%R?LSt<>@Ou8(3k$uJ!WAwPhXtUHGnzBbi!J$CrV_<)1n;R=oJuHLBrTcaS)BZ$QYbD3$y_*s1PNi)-e zZaTdxgS&>u@NfK0vcKCziY1>bgW1L=Ef&4F{ZK3Q^gmhv=1a&|Djs2yFM$i!nDLPz%C7cE1;NZTFf; zDcq-3G1nNS#Yi*W+*Jl+qjGei=8Scw-anXl*J|0g8C)YzeE;)n=iBju>^^>og}C2T zZU9dm&HA6GzHxcI>0uZD9T(EDw*pf>tu{j@-qkrQgix?J? zi^!Sv@WGR@YBqPD6np(!%-_O2$}J6x32|@Zgw5k%C0HVmQ_^wCTWRC*{5jg|@1gr$5d&whCE=}2bA{q2V zCr#~&@*Lcuqp6)ySZA*>(2HqBVWPB-mYerq^M}(QI!I4szhcfdHhVsC&XmM^eA6DQ zQGnZSuJ6S<&{BXlse_c%%K@HmgI@7--^Y25d%@GR-y_^Vq={S!$_qOCP9hihEy6l8 zq@>I~!+?f%fQS>;(2U%PqF?j1WHp4x{9?I~uJR-St7qF#x1Pv>lt8y?i+TUwwJ-_TOEY6SRs&!w2cjQ8DEz0c!(SJa3G*W9g}`7g;h*p<963|ehN zx_51>etgvmPg@t+{$@tYTElc}9b}1xUlfg&3)zp3XpM80I#yW^TsA{1QSY4}P7+;) zISH3;^&|^+6hJ2__X@BMq8-E8&$CArrscb7oCK?GJU*1qIsh+-gM4I?W1LL)jo}Svq2tlg=QtiyueG#=UEp`#hw@4FAnfIf zgIM7&Ueh#n&eT_J9Nxoxg|tzS{rw><-1_C+4lwbARk#)$pL1EyO0hnU!R?d1(`N2abDV`T5eD zMA3?OOb($wU{I8AB#Fbs{(}%G9Mw=UKs3e|REfn`pAdP&+*4tfg|7oGAk_YK@P=k` zMF;o0`%+sBIr4M?_BSI_Qi|qTzd-NCfy>|uFev)HJeDdGwyCAOz+r*7z?Ro5anw3BVJ!04NZERJNOFN(4vP>1)azjNzA>xyc0yqoC>=4Z{5c* z#()X(dC+O<7n^P)&gh#}4$dL@4)jC=)m!EZAKZ2RK)zv`PPM7Evz=+g=-(i7EI->#L3@z?}Q@2RV9E>@Km;5vPnsG9$L(zmO@_B@4 zJG4(Lrbf_;&A*P4MAMFi?LPesqvu{8)nwyx(XKd!3St3{q80LyofyRhg{FvlS%RoBN*h?JMXJ*lQ1A8t8g*+F!$D3MuCb@qK<={$&E<#4K8Os05x{C9gUjvdmE>mG>`)64xVq=ce5$30RJg71v}s- zqf#w>|7{CDlIG|t%k?n+?$2|HqkJp@^j5e`LDx1}8mE|11D=YO0PKj;9r~Kcp@G#< zyv;w3?=?psB*HvOvODUrGGGdoTOs7}i`I&k?REso)Sz$c#C)oYJ`hz$T~ z)#$Sq(wN@OPDvzc68u-u=hnE;sX|Rmmg~+)Y}&(i#t2!%-snFjcV2LV=?-=(XknQolI4uQ z`*|Bk{nI;&Xa92tegZxYZCUf*jscBWVK!Y*++6e@osFA5qYUM- z*eK31Mgl2m?seF%%Fjj|iDwqv925godAjWjyQ^X_onedDFYBR#iN)>77-c1kb-h19 z^PVWFfrT*rtJiobecS=Vov$uKO0rf&7X@0?9pTOl5qt7IR@1a~rx}q`-9@}^W1_wb z;&pN3kVIUhG1QZw*tsFGSg~)|DapPxuZ^e39#M|(F_j=70whr8O z8i`v1d|7saL|5PMbb_+UU22B@jzE#1PcjasJ_LRBI1*2dEaxxYBhH;GkkdrH;d|Ot zm8AWo*zXto#Hf+M>ke>2#S2fWO&_odGowi9r&@4i`z(mnY@F`NrA1^O^5gqQU465(Cd!P`vX4pMp_V94 z3TWitGn*ai49B|d(13YqGIB4;;+rVBo{nN^y<8G+CB}wwbVSn6V0lFaeY52Sq!z4b zn_SwISe&T#xZ)woRAny(vm*9(h6`XxxV{j4IxPzI8cj~%P+sG&t0KGBzj^of$KX)< zX^m@9()(0CXMBRPELJ}KO7bI_$BV{(Xf?h9;tc3eMxN8SmvL3e&-@Wd#ukE(<_!*> znpq}^R6<67TmRaoT?NmvD=2h$7 z$ozC!EzWfQPkK*vqaw*}i}Kh@*J5gI$%V}C^_QZX3rV5+&t$Ftow{>gpU_)1Pa{n( z*g&gkQ`hR7g0{rR?B?8(3fqg-pLBG{>wvMlICW9vz-J1mw7K zkgL4Hd6A-Sd4G&UbiI~ejWW@l{TI2+7DYR7cI3BOL(nOa`CUWuOFnbteBX^GfeP5C zG5IMt3rv?11IKa5{bpHQOF`0Te^lzw0n3H+9Cz9^T(zx7j9T`Mz5ZU%D}i8_b<$vp(-`a*af`ao#1q24^EDMuSXH z@s-^DT-_KNZTEYk&+9wo0uGVP9qhVVvl zkN(IIdBiH5inkv}W2wD`ufF<43h<*8{DC+3CIW>+Zy%%@KmPo^>}vS_c5g=;eMC01 zWdI9_80Ks8R}c6o65&Os|ARX`rS%qmn%OF##PA*KKKrvD?~4$5tj%MnbGDLM6;QJ% ze@O8tglmrgH0&d2_};j-Rc~yl=h@E+YJlNqK7~w?6E~{&XfrgAACkYv^E7@peMJ3r zK1MPnE{vhz=BrmY72x|AFo6Tm^2B&*n;QVY%tsbCl^A&88_4@(i*VuR)sUYz_5rM) z4vQH!K_=QNO1d2Hvj)q%$G61bC1(-5{#;`31I2iN1sU)Gr{i*P3r0N9S!o88kBCJh znaKK+;eL1TA43ra>Kns!8QfJacGtZYpUAETGMi*#uvubF z|I-4e;ok5R8{xg~0p%qJN|laV1A3}c?Qy3&1$T0Q-o*}rMX?}X!?dGBa6uK>PMfO~ z$?*E0{>yg4p?I39pC}MkZv#ESsh;a?my&=1Eyt#j${{r3;7gGVdBZbEB3gl2JEZFJ z|JePOatQn?bMT$zz-4NOB*s~oplh&DPZPdrGQjs+v2JlPjD1l2!^jhhF{2vKMT&bp z$49f~c9mV42IUwsXG!K(!j;=>K!duYLN~YOMn?n(!r?8y9;{9ulI>qYsqT2S6;6N0 zn*R36Y${x6Gz!9z1=D$8Y8m509b2r%9U%k$hEdNDl$0QpsG1okU~L%`MNsix(=9P9 zXtJvu(Cq^<)s^L6ygw0CS2ZL6DLB()d;_(0kAGTTXK3gA;YKE7ulEWjFdB5UB)_R_6S#)=raP1R-q_wo2z<`h_HaA5mxfg#_5FdF zIYTNtg*3n;89}HUs+K6Qd;~fr9Kj5>BVs)V(EN@TgYJa+ZhU_?Pmr^jY{ch@h0qb1 zLd_22I%}4Bn?_^VcZ&9r1Y zGnpsiXq-HYP1yHHtZ4)ba zfM=TZDwHorv-wJZm1MpjGR6zDPVU}o66eCr|9G_<&fzRa!%V&e@74_X;C?UQ!7D3k z1+wZK*>x)1sW zv^VyvDja4xYkn+R1c6phHsC|JAoHVVNlb;3w#WB#kFl7&@W=e_fZOlO>!`q1BxcmA zOLY8n!E$ylHQ+Fm=RF4`*@A(sgi%3{`!(c>Wb|p^PB$L~2$=j_+WTX;p`Lqt7G#M* z!R|3Iu+ul6(fOd}6JRRF$k?-n+(msg-|EQ3!Jx-n&sk8faB(-}!NwNru z>c<5?D~+zQVxPDH%Q@2+cZQ>=Fcz?;fd z+|j>`L@uicJn1Saa>uUjwU3aYR>WSD*1673Kdigz@E*`D*a(G_=BWRm6svGYNzNsm zftNeABv(G1PzmLeAUeQap%1<8(Y;`}SLGy`^_|B7j8f()9ZOqVh`KA!n8I zWQr%*m+qpoP(p{$( z71@}3j8H|zCgMRHy)*`BX``g}Jzvpy;KD%Y%y1R%Ht4HvLkPX0lsxe;f4CLoUs!=+ zeOZL4$N?!w`I@!GA{LB+@+9C%=sZgrAVuS!dC@4DB>ql~b2NHuz70?x4y5IpIYRi_ z83HN>;K37>nskbMg+Sx(4^J*}`1xQGWCx@+jH|0pQ|`sKN?b3qrUL@3_6M)W2)08J z3$7`OzeijLzZ1m$D7cK-Qf;74*Jfw@H3FUh{hts369)+asKNzu)l=Ajl3yeaYLR{A z=BicK!M(&tY=4ubbNhuc$a7!<{p#z0f+)lP{{qxWYj*mnEk1^v=K~DX$K(h2tx1^Z z8Us|&8X20FXt3Z4#9gHTXwG=IG6udKbSJTg?OC&viieUggBLskbb0R&iBPc83E8D0 zYL|R>n{g3ucYB(%T43Jv6{ZA;Mm zz60cc61V1pXC#E)Mdz%nN)d~Qjmj<(BaTEzl#a>eCWH2AV$jkn@{=7&@ zdLU|ROhAvkQ~&H>X+}SU6qX5lR?+E2)i`9-YQ8%@b2i>pLn8%Zm6)tu&iE1_p%5MGH>?=MAz zAswbfHY+1O5ypRScXg~BS{L8Fr~$@a4m44$HPs;PfGk?}gv)#4vW8KMx{RM8?z5UU0E^ z=Eh~MvHbI65e7cX`B|EQ(|&|W zWVy^XguC!Nzn1ePc2^m+kgWp>(!m@lp1sqwzw_bm6$E@nniYPFOE}rs`t2o9ol(@x z`OO~-T{4QXtFgeZ)wceU&qyq}o4I@-r#TB0c}{&RFStN+#X%+Se_JEM!LRLp7!V2n zI#msVsYt~JS))(`X%;Fk#g$8UYQfewzdO)JgInG`Vme{+7rop-oP&Rx-j>|iX~S#R zk@SdX8`1M(9cEj|@LUf@5*h~{L=%HZY!QV-&71LZ9ISor@VJ$&&2$MQQQtVpnH-^Q zYw&!7VONS>jdIVHU#87yZM8T+bC;ZC@O8{`4G6$Ue9Y_=M@(hrUE~x#K5f~m?mfel z5t3zYBP&g4x=_ahp^wMJ&1IqOcE5UoY+eaWw9rg<8Yf^7vY!Sum?l6a+svd5J#K@x@4LsaRex8 z&q7WLx+uO_&kzUueW_yzo=7h8C{PG95-XR3xZ@VkwJdy^RN;TJTt_BnbXG~-;$VOC zizx&1HE9gg74s2(3TSLNBR;(K#NKrB|Ez!de>i&!s4BOpZS+u*(t>oCbSNR+N(m@R zZ&4aVQaYqX2}MdmQcw_-Q~?R4B?LqfQ9=+?#|z(RUz^O^+dURe0g6}|<}d%;6teaIMTM=$(}HRo%I;ZXxH z%H@oVBanTFA^Oh^+=OmmI+0sQ9`XuEH#0!3jTtHu*GG;|ewu{j=S@UlHsc%y;`zysRb?U`DDh@w;&MkVZ{$Tu$0x z4fsBxXL91o7(WoL?n1e#8--=|neI{b5LscP9;kZEMg4*D6yifa1uoSm9sedA!n%>G z6~KP>gSMfz5XiqR(qmFt6teUJ7e}+!^@0SPz01@9Fd}EGJfp98J1+3H-#vWuM^FO2 zAm{U)KPPA--38j6+fqkwLX^3B`9YS?+q^|EwReFb+>J~k<%7*#2m;sE!EBjB|Kbx! ziKY-j*A|NnU@fj&9wkp+eT<4vQB)Wq9sR7SEfcb~iQOLuvgD z@NCUC;bJ=d4|k_KRrFf8TA->|BM1tQF6KtFf>LzgX)BP+&LYo`n(C01W2g&@im|lHiQPj1K;Rz zaG5z1l({NUx{*ht$&S1TU{a6tX;hP@zo!!s1dOL2Gkg$05lz9C?P;+lVk7e+==Vna z=fx5Fm`|?aq>z^1J>r7_VC3CI?83R2$Lihwy8X5$0XVuzFKI*i|Jc@3#@eOidk4SeoE4HIPdqT(!!p>rPyGE*^-9xYEA3rgOR1x&i5 zPD%0EYFj{IpH04Rc6F+LYv$gS3)QJb;eH7q73BSgU2L&#iu_j~r~;Yk?m|<*$80mk zHk%*Gl79X8cs{2uO}zE+D#VA*N|NfkNOO4=V%Xus;do?Wt#7hUjy!`0vQ$5}%$ZUm zGFudT5fDb0gpe%hMHm&d`6Kr4or}v52wj-=>^@&cbZw}==BMdd^F5hy3jd>jm?H`p zYkVsB?SV<>PjwhF`|OO@Al5&L)r7bPbbt3D^XpX|Y7|AIUE1wo{E(V^d5AM9&gKYnYt3X7P`*!!04M+8@qEov zC*?{|G?cpqwbB;agwOxIe)jV$f#0_D8dMeCNoHQN#b3`o%zd$REvjSnw{q2%>~X@u z=F=Xfe`(2+nIfHS|K)gMiscW$4-c|*wJc~Lq+tDCyp=I(;L0(L^q0DGcVk@9Y;qPJ?(+Ts-P!BMe?FO)J#ySj1gUy~WP<}WT$Bzqq&kSs4QcbZKtR3* zC~-fxqs;cLJOh~+_4WAq;V&1;;%hfBtKO?8?yIZu4|Ew~QXE<S?CQmHuvaf8 zCAc*n-EKSC>pWDP`vB13Iq%)3o^(k9x0xsI zs7^J0`4g|m6uim)Pcqio+Q6jb98pVDzKXw;0RqgMT#NY9A6+B?PGmK{eDrUyyY~GG zfZ#el7||0k4DxO>3RDK93| z!!5ywHs4$TxjEAAZX|kH<|Ay%%iY4YJy-9JAG|QoTlVV~Dg*X%M|eXClt}4CXans% zz(aW(=kGM~I7@)&4=SNxf53Z%mv(&>C_DA-2c8h)9nP>ZVj=5Bi;AED$iRaZ6(zgOpvF^9>$Gwd$eh%8o6vNq)B>? zO0&l4W0y3MANi^06*ENbSz0v|W1pY2F<~I-NjrWtZlAUQ37vfm^^=GTPu@d@=%$|W zll%W}2*?Eg%K1EfqjHO+QKIUbp%mmeG}PYQ+@BF`cENuf%*vFT^rzSj;P-yGm#$*R z1&~!rYzoTsejDcL5%Bd*#e;7`?2FxRD(E=x&W8JW_}pW;evE#QBdi|I?aZy4R0mJL z&wem*DyQ4kPq8D8ieHn>nS*D?Ac(i@m*LL^-RZ!Icv5q$WD=M)XFV&Q?*9QQEF}Va zP$QmIIEW)ae5>ZA1MUx{1>T^^n01MGvOQ6(~^)9Ayr|j_O={q++ za%NY<#Mr-hGPM8T&I3@zo#?3JH<-ZZud6=V{WlHd9y*up!<;$27Soln)Co5L=UDlz z11U>jck1RpERavN>TBFMI`CF0Uyk1Tdy}C-3y9%e!RMtXgGXazB#s=sa4|osYs4+~ z-SPamqNjMom9niCcp{E{@!|)2lwJE^H9J(rg{dpRJ9JdLg-zV38D_$2v^`tD;B&Ca_Rq?x=Xlg-=>dm-ocEC` zb}1>H(I%rGzsEPr0v)ZNG&0#(`NAfXW{(4e4XkzI(Su=$t6xZZ52(`ru#~vjLExx! zWVWO5!J@i6!i$ph^!H0%I(TR8e~A{SPx-3i9)U$PRSCVhzr8_&V}DRM&Gr(_@L;kO zK4ZvyqCrFwzelLLY_}y33(qWm@;ACH|#AZa3gb=6wiyur7v=d zmYH*yRTnnhGzqAXI+#u!^kpmwj z`uDHG05mda_cEWktDT)YQ4;1k(@>oK;EdBIAKcy!0_ifu)5@?$gxX~Lr$rf;3Q;y) z$7LA?wx3)O2a*`WVAQ22`$w1>0lxP~^lIR8q$>LDv0%Hux4Q_|oX$;7sU*XzYL*4eHB*2d_(nT!Vc?Wb@gulEkJx`133J*mH(_ ztbLZ~fWyfYPDJ85NW&mws*RF51`V=Wa@o&~wTCe^-0Z4hoiG007J2BH!18J4%L!=+ zz-AZv`pzDUKMDoW9@y;U!S;_cLsY@Z*o?hVhiA71W4_LJ2726lp|QQOzl@FlT>Za< z4YRWmapUEfv`JrZ!1{1cF3$Y(&jel{&_LYfL4Gu13}!u(Cvf-%;GGnKdD(x zJ!bik{p;H!l=$kj4c9R$q9;nvYc%BHo+$8DNzP-BxsFSMYw`;BMxN1G7j_mE0$TjS z4y~kP4Yhw}(mn0uTJ`6KuC@#srs4tcj3;N>6?T0rwbrk38>xT{eYxL@E9_=96&;1~LhCtx%NqgOET;2!ojE-SG9T?2G%{x@ ziiKGFDkO==?JN}&6t%~F%5*kherlh@hRZJ2;F^8<^RMQ;5BD^7V?Hh zZ!&SV?q1#cv0;~_#9V%VLvA`@_z%_NkmX-T33^L!`N7i8;qj?;2g_O6Ob^X( zV&40D;@OWIgOgXT-6y0GEh@dfymHw)%WseO4vh&vkHzlQ%=BN2ieg$5zL8TmcY2)p zJz{p;sO6DC1R2Py`J^laMfaZgh-MIlb$0%?O1#X#o%kGoGgJFhqir|XuANg&h*Y^| zZ~gWddLjFFJTW}*d{gl zD++1L$A2%5Zcc9ABS?VfW)d&SPV_`RK@a-EhZ~Q(`lr06vrL!8O|3 za7c9b)`^8-wA!xNm22{neD{;UtlWw@ZbW!sMym8}6KUlIkm?FeAo5+$Tr#}lK!W!1 zW=1HJWts%;YLd96*X&VuM}HE|N|n3Y9tLqQ^k`qg>Aa1|gVfhp^#Ew2eH!~Fyt@J{ zS9Iew6e*K6E$l{|UGMEht4f~KuGhy^1P^9zl|0W6#w(x&Q{uIT;k}_-B^obnU(a)? zk;(asrYOf0_C0CF)<4C0bJ5xI&Y5?mGYIDGwdbW&w{F>6;{i-2- zpk7AxwQcdWr*n6NB|fla@EYT4qS0q7OZ5?dmX11vO}0}1XPq|HS0{;R6ZNe+fx~@k z@@*We4Xl_i?tnZWwVSXo*3X{i`=UwDTi8UlW@X32j0bF0#%b%PS4x!AB;euP^k6in z=S9uH6-?a0m#0a~R~+T07(2`HQ=K!&9U2$X{lbXYns5#f>GHWd&bapeJjvC4A{eBH zY)n(8)@@$j=(J6>obeU@z&X$cg^08Ff{_k*@}mmG1<> z)?&z>AgUq);^Fw{cf$vJAZUcuorWS|7l<-t+wOvRT?Z&~uB`*M{#5v(Uh|7(rj?pd z`+GQ;1yFS(2;_yjrz5`+sc&b-JU&~r?-6v|2o!Y&W;~F zbZzvvgxJf$F+F3=#}?1lF2aQY#nbB{!^P^c)E*EArWG{qiky3x{C(`1G13?ZM9`bd z3m~Mz=J)$Yt@jQHZV5q+w)^G}(ez5$A z&=cWxATRTR1rmG>2;#6IOaA36#D?|ECw+9=1;YfO0AziJF}sdWnW8Abrw0p|xI$F- zHd6m|wcqbjg#i7zh8uCOByq~~moZVR%67Ii_1Q_&=UMJ6oOxtY*7dOX{wB(KqULL5 z(B|>b>nZia&&)tVv`Rl)-ci`PaV!JEb^S||?kj+zFhlJ)s{PlUf)4K)xAYj{wp|}_ zfh@95;FspzKfk^GPWPBXTcBD|0mG67P&BK%(hiGusm%K2cw>AxbgDCpBsuc*bgGCe zRL6VeK>4v0^t1)}^}cjM-m4U;%ow%43>(;%5wtp8Kfk>?+gJWE{1qlFfjZYmwwT`C zyvAj~J&d&n>dR&jgja+7rW+vfI(lh*ukLKRquzaThtu#6RWaET`nwyFPbM$DJB6o^ zRa%%Q@@K=DlfspA_ z?wE6!QSG;1vUfn|v<##mFJR_q6AyV$C~E#izvNCLXi{2n^<{bhK|3ZlqSkojejRWl zj2^Vsex&U5Y;pb2v+rvrrh{pV@rz2oe($=(kx5R@xAf6eF-L#KpTH;Sx$0L*t()s7 zR$4m4I8QIM!f%kh4!FQW4IOc`0(s?vqA9y4S}S-Pp`<96Q?iP5LFvxgb|pXQehrEK z5YmtsYBGgK&*FU;$+QGTb59J!FdOqWsu45+spIkLIOK&*0BF0myfo3OcF#dTw7?rq z2H^)^fcrXzIZ@y9S0#|ePkchvf{$}EF&K2hcwel85b3-O*ZZ4I2+UL#vNOfD&A6gW zq{g3q^-B}Z0w4*hPp1euJ_y6?!ccXoA3+6XJ0pY7sXdo0`8+7qve@JEb`kOoW(RJq zn1+*h)1mDS;xnv$|6QzXo>h3w=Gsso;# zZj`Xmi6hnZqlda!hnC|#*l%$3Pz@y$u@H)X zth6@>ddbkM?TkMIhu0>{f4*cpxV@pQDE@*IV`y*O)1-HSnd)};>(u3Z6+T62=g?f~ zwpZK_E-?EZ0-4B(VEd#-nVZ@Xd^D_`+G_WINxTT0v5ieVX4Mgv1M|0V%&l(X&>y2n z`gZ&(96t^@Xd+phe=Ct0V4tVxXx(0sat@E_XYJh(d|Wa3@JO#5`8XF1yVDubCBHmju?RoSxc~og(%wk?au*nwu5uD9(j3?D1uMbZl1=Tof7NMx~ z))79nG9$P|5llR#$;1;uX^Ai-GM&?qbrJ04UIh|`b zB6HIXpCzhiB0Xw^_GXV44w>_bK6G_h(>W??vnuY%a*L3VwY3R@AI@<5I4+In zQM_>DKqNV)!=(BezlUCG1OIVbl-5hfS1u zOExeZAr7CMlJ4k}CX53+&Sk>I{w!ge;sZC|xt>6U=~zw7A^%fJ^y9ii;+UqtGMx;# zdO650O3yGd8WzwL?`d#}y;&o;^9!Js@766rGFPsPNVoFNU-Gd_KPTB}is?%+r1GV9 zJkOvN!vKZ(L|KIN`#|=7N(u^@>&x;scm2O9ae7@R8e|uKH3{bDa&FE9)$t(5`VW*Z zikVi=>xEvUao;?e^5yy}G#+^Gqt?Oo7@*X0g7vp~*%~>78t77s`7>rC9BGjA2@dcW z%|FifFT1@g$qag}Vhq`A64{!C=KQ*pXjf@e$HXZK$DXVCj)*h5UOkYnwZPBE5lgDS zoJ7(CM$*&{hoatpDeoL$Ek^ETHAV3C`#w#i$QBP2TcmJp3G7qmAWg!35|Qg+5v>e^ zvF*Rk+lsroC6++bLWcWMh*8{SDT^k|apw2M-eO`(JI%kE$%*89M2~au;SR>|#;D?R ziB0^_2>qg&wZz!!95>aQB{_4b_4f@=F0ot1TA71{-zk?UNRq>R(!4gcNQ*9~QJ!MF z+FV7JZ1~7TcmZ0m)GRU(>BeA;gYS==IEy({Bsr({a)L!etX{Qzig~1-2)x$%;sjgp zN)wfzqY1fVg2}5?i{y&xrHZD&=kLXf9^^Wb%Kx#Ry^=ajJ6wZRoBrD$m3oG@9tlUzsA>{z*Hcr*T5r2gKik5RQq+pt}CtuHTf2~<0K$zc~zlKop8+InJNBc^QM zwoFB@g;~7;Rf`CJv0K+D=G5?Jh&Q0TO(CAY6G=)x&wU{=+_UG16vrbC#avch$&q2Q zaHg=>5(1(Nw((nWl%X?hbavwLB_qIygC@!sgQ^tPXr8w)sZAU`A#KbiX4&dXt9SD0 zX}o44cVclie%&{Tb;}sWgrd>lPTG=L`xgB`-Jm6-$-vPn$Mo;y-?eBbxg{E?Q~7gn zX=sn4Y!N`>z$xca>_|I#RB2Gs$zydgG%e!l&1ss>GtV6>{hFg_bT!W$O*pcWp}~6L zvd8AqMa#@PiLVGA_;QKib$c;#N{{RKN>F`Kks#H?B`9KOqsW{^j5enyp%;zGvIv83 zrXE%$$6=L=5r{cY)0257*;SRhUe`>>OV!^c?$Y(q1mpI|$DGlLC&tgZ3`0`7Bkr|Y zOtd|@r!n3GHdHN_J3JH4nRa60BR(~+hR@$x0J_4;vUXQWOf}*;F8R7pv5$W#(RdC} zLf!UA5?yu1Y)a4hkx(zW9nrHG8~VSJBe$M~ng4ZBUv$QWGgR33lEVjIf{W2}ZarDI zxDU_FfCt$_>Bh*$9UZ9^TAFn`(cD(ECzq(^fZh zhNfm@shrsQ_Kz1Qih<<(#kk7feoU72b60kV@)zWgyG1W)rnBtvnlH2I)oE9vQw6Q> zq%I8Hbj4h~64h>4ekZZ#n4%GMSgf3IZyi(P8a*RTcRB4Uv;%es*xl+2nZNsG(@=|z~v-r2{1R|YMkdkno$c&-Q&;Wzx$@YI;yW}=Ij3?Z(L|d z#iF=Loq0w!?+Vvbz-HLTaa0~-+L;(uXKTuj2^Rmz*?i}wN5v$YD&4Bg8#?7&T$(xd zB{AFkpcyB}_BMmSTpyQ4m0@cM3BEeo@rK2czBeMCJHs0xFponhX2KM+@JHm$Ri%g( zAkD9n6LjEl+?qSQc{%ma_!-Gp@v_;QgmLMrVNVfE&y{cE@@rYev_qj~RX6Qffe5Mk zKI8YHaWfE?TQy2*`+fr>E#F2c>!<(fy%gPkc6><*g|3seyVlb*j|nGAZnl}volWtZ zi=s*EAnc=mbu0aRjBB{+uiG~xn_jhjr?j-AqL}{y;^@H>t*kWc?Cc%=tiB^XV#4a? zCe1PD0JIi5{Jz6^W>qDoNyn*%MKwd)k3-l+TCKW+Fo}?YF2%0XXqOd^lEWe0eO)qdgS#7n(X<6^%vY<&Mf=vSlA!)sakQb6pIUfNeI0$B`4 z`g#BI15R_V3!IWxj`zo3E+6jPp}Fc5oL8LdVyMdZxIY;)phQqI(-B#A-tSA5>uHw= zJ^G({j4SlVYefeQ?4!DAeli9d8GdS^5;tzqdvI=2tef-h)!`a}ftfT-VXJ}CWCvQF zI_AMN?W!Y6PfCh#()jycUM#21$di8*Z!36F$Z6B!*xBMQzAQ;e1%}3?UID{PFE7c~ zMGw8xetNO>P;*d$$z+@vCHLv~B)hz7c<=8oHi=Vlr03AYet9j2P7M146`hGyfMNM7jw87TAel|EmRjyq^>SY;!mhuG}8*3dm)mmg0dm{O_E`~q;|2xCw0`gI_2Lg*XE*dT(fFaw?nEkJ=;;d@kUc((DtT$kvPt|arC4z^INbbV zO6gD39=jRJtr*Fl?xc^_VmBotE)c^}x#FKh%rVV7stMX#0DQE5f3%?gPxZUgH9BS?a+@xULNVA23R z9}M}U9S2uK3sgSI@UY}QZT3UxmS}BrP42Drgu?oEj0Pl&rV}BnMUEqJ5+xqgQSJ^B zRhp*d_MKakOJs$!1X(ifoP&MDbV3)-K+q+?b2@pczR`iyqQ)h?CSNsjbMKX7RgFYY zFVT{0l4NiX*56xZNAJ-$$}w_ciZjR8OZn-LJ49}CA8X6=pB;A^N<VlJbedD<5n0NRAxi*0@32$;%_kcM_KRb_1a!YWwn(3Va6v zu{&j{G?&Y%(tvPw2LNAk9XF6z`fVPaX*kNmsu193-R3i3l?5o`4==#Iqyp&Y)Igp_ zp>i0m$J#f;50&-_8~!bL&m0GNo28@bR=;-J&whLv@{(EBLmd#uJkjJVZ|B6MtQ!OU zI`0amW|7-ShwSbcm0l2MiGGg3>jlERFy(!Iyqx~4-`|T{io6Bkk2zd(01xqjggf~| zE-AJ?8nn&C$4mub+}qT!%gBS=!o3`g1%7gGO~#yDi4i2@6PpFz9%``e^{rop!kB_p z{sL(`^b{Kr2|;r2fZ*-~ti!R{Owag@K$s+xxM9GUcSB*t|CuKU>F{oD`=BR0-`Pw)Voi1t~{uuD{Q5&nBGX zt?vtm`NXtt9_4`aN)mvNPOr?o5rlLine*qs)i;pJJJJy20PW$-LRDipl`*~Pl3G9r z=ej`UPh7p2A!iuQx^X$W#lD*p1Wr=@J?UF9GW3BiXvXn$Y-!ADp&# zEwtl$LUh{zXsk{wY#MS1`lY1E7dXNrg;jr(pt?#6LXLYp=cV7sC#MP2wo}30BO7X+ zyxCN0{2*Q!8nyj_IqHB+LIG?$)CyppF`p_8P=YU{9QvDTXtTA*?q2&bdafP1vTMj6 zwbBa3La?PA->RF{UMZW(#OJNg0_Mh(*+^A+kVpCG0a(-10F}H9I4&gG?vNl*pTY|= z*r#TCvvi;&i$Mzg*J zZcX@Gps~G!Mo8afzUmFleUI5=;a73I#ileK{}ZyT{m-axgJe+t_XU|VYBSGEk;lCW zc-}tM#{MP9zb<$=5h<)u_XR1#?qc)8)kC{W5Ox}S&)0TEVMy;tE+U&!5T zI=gx_oFp_R{JpVY{SRNF9@s%|6x+kt2xfp)TAg@S?T<>05$4du(kJ2^n#KicMnp@V zL`JBcV?|vyKz8je|H_9E=co>~S^=|{!3~%sgD>;nQKaP>Cg%2*OySZ?-BLyhOO5Of zjukkSpY>!UfxLC&4qybCIi-~YH{Q%f^vOlcpim|h4@WC(>qF+t>4$N3 z9qnP~M>TDE#BlIedCxz#5O5;0+_RX1OgIl5hFkxm^;$4>+6DZGtvrB9z7rf;zJpk7 z&hLai^;u~A?E9F7JDBrF^%2|yMyZej{KhY4sW-D0Drn8js(jOsY6744gn0X8Iv~&y zceF@avwda(a*F-$Frr;R|Imhv=!{?(@l*xJZ`f9k&bLf4f8@C+s_VjBK6Cw$>vsGC zSu9`Etor2dP&#;x0($J-BiZmXsjh}DTV=;4Vaq$K&V<$0ma>17hi#{;U$DHM6U$s< z*a39?U`UWEx}}Wnh1%n;MqZQd$iKFf8ff&}=iU?)BdDWZ-`zzxw+ zP$1r_91hgqth9DGg04DzYB=IK5VDnlaA+b;f^phk_;I7#nnNbKMl~*`f4WsSJ9Q3) zK(T+#{$)R^WDW740D(kT`qB?8%-6V9(i=kIF!aB2Xi7f4bOAPd=pGRqO1P{Sw9H`J zP$J6M(E~QSIQy6MCgz=uLvud#^joj)CtN_hG;)LsqGX4p0{kQSz&A)<_fN?bCGR`) zl$~1J6ug51myxFzk>jM^WQYkCUliRMiQM7Ip`Eus5s7bv{z{h@4--`36S@C$TWbDa zxd-8Gk5hl{ZE2C|l38S zEcN7Pp0iz#Eni@HAz(XnpgwWC(b7eE61fI48>Q*!%bdu?d71gr_;KC+%LTT+De>r4 zxAWMEy1^D%ccqGo0u}uZs7$G#_^Au(DLmJHjx|;!X>zln3%tC~0fbbQJ^syxi1sfd zUu`eh9a?^oL`i7fVk{*13b)!2vNk1GU;D5K&t94BLKO`m$r87W8s)mFuF4=j|M^{k2qAOLiK?&hzN5Kq*N5%n7v>G?rb(%cQUxx8qPXtX`p!7!pRSw}hp*?eT#LWh1bdR}D;u``4jJ zW>=ECdSAv9F%Yusf#|{8H4p5xbI|v90Rl(4$MCrnMyuLnAg|@`KyrAHQO>JaF$cJA zB2&Ort?51rhZlhT$C1dH4>QH>R5&u7AFowq!T}B|9?Y4KW4& z8e$aCiCFN10j;?e@cl`9)p-v{fB0m|z6a+Ddg91>ygqUs(FfN1H-SCz6`1_Dl|q95 z94@~MIl*ZlQ%HPRQ|0tM0XStm1~E+NO*SQ~6wvd{y99_$ab+^Wpf5 z-KqTS0AMlYmH$Kl71&)Bl8l2BoBIR4M(`&0EUT_5;7d`WKRzlLN?&>PDS7p`P*-Bq z(P|8um6%~{tiHk$EvA*LMv_PbY`7>0C0tyt9U3!;8sY|H>9iqe4p3om>*0C652go-5LPn*qwJ*PLJ{qMo3b7evQW**#>6(F(O^}jKfPk_hQpY zJD;7?Z%E2?cj_25y58pxzQ^}Nkj);G14>9Ks2=pjlgeMHz=W*>W$pqvhn4xm6}8#M zCF+rdCK2XN1kVej#(ylx$oQ`n7e>AH&NRz$IIsjX8?ima5jBEI{+*8F1eqx!Tr$bLk3jC(eOJeU(s%54A0(+29NB_uQ{b z@>|Hj1_o)yA(X;BxamPReJn_S;S~E1KwlJDrdBqC-67^hSI|=1-l@I-!S)7O@QGMt z1-ZitxL`uwdNUk`0=c%w-+nnW^2`0+>EL#%LHs&BCujy5=%mg#FaXy`ex1M$I z1-X&dWa97Wz)k)#NUa;}iP4>XKjYKQ>VU9hQF2lKr(_6;0hgH1-cxTF#V!}@0wc_J zIh!LW$|xU&SnuMOC2DjfzX}C6pmJ|vp9?r>bje(9z2fM8Wx39HB97~4hr%OD zrkp8zj$yY*9SHCg1XBr;l=o#zLllZ{ql0cW+O#WeqYVy>*34FCZn`mmc3~o7 zP9aqjbj5+G(E$Z3&&#iXGg=aIX$Q7ypP_6|W(GpQya~2N&ejfk5+GpQLbk`B!re0e z87F#V1*sLP4w8nU=FFx*(V=_P7_E#qg%!i@Fa;E?as~~=c8Y(t_f9STo>Yp>7LA{N z9=>e;`Q8cF+t5|JV`pm>v8R!Ai|~oo>S?eDD_~%|0G?h5F8$j)4N?J%x(X1U(m{8C zWgEnK1z?>$;4)sgcrCdWEO@$N&~q1vivdQ#3#|ZBpj|uuIf9dRec8egCMUsTI#J83 zt<~97P=fh5hng96%Leb%iJ%TV8^>8UbHIA^E=yj;(rd`I$d*eRg*|io ztcaRpSEac?u?Tg41hy+3gMxn*-EBNV^5oZX^xbfz3PBk2`o$G=j*vLy+)&9K!~lhM z>3Odmfg5fMobj4vxBBHMGSG*BR=MS~VIpAjprsX0vHiFHT)9tR^9!L(p$Fg4?n~=*K zlC;#Xg_i4C`hJpSz#wmV4!u2p1fuW^+1TIAgx5ZKrw%{!v_Glu$Kee)fQw)gO#q-$ z21oLPBS^wHmTcYkQT%$mI$60I6HW|rHZNViJ9`P_JjIfcXX*R;`MN|}U zKMVNL$#IVvdvjS|l)0ICd*!-~O;?Op4D&kycY*Tyl^e_o{%8B1xO{qE%`aNyS~tka zR(fU;vU=Gk(@$QzA^?yIOtu}oeIW=?KCg!aLN}yK)^>D^--F6xS@ihbIxMBQ@%PV@ zo&w(jAn5e}rpSMruMsZ42+95fLH$~`M=LJ2qGq%H?i6%eV~`p!!^FeI?!WO}0I(`aD_^@8 zsy40fH$@zMJ--G=^h4tna^NKLLDov_!On&_ocF*nSt$Uev-1!GNF{h>#^YPJK?7Bm za1wb^0+V#F5$TodB<++nW*wK>CoG87elxEMLQ^G>MXi}4XOSzI)z9EqK|!cg81@&p zbwr*-cL1H;FKY%YMMZaW^F(WOc(3228*IeQ7a^Y8alKqAPTqBpIup%m6CyWKoA)1|0X~&I7zw=*CA*%ZI%N{ILv*Kn zWaXEn;njCODODj6hjr|oqc)99-4e0qw{FrKu>xH1Ug!p1`tZ#2!MHwAfThiadT?Pz zeZOD(03Em*BKwUE6nLBno(VyQAc}(RaY#IMfVE9R4>%rke^genM{tL+ROs5IxJP`+dwp3=gfx2yK=Sz?(ZuaAOYu}%{Su+{&7QIjVQ_K za+8WUGrh<3qy&~+HQ<E<1RFgN{Xh+0fKZWN%wBX{SdWP=A>a`X)9qWzj~D<02ukykJ}dDW|5-|B2kB{;wjpAk96FKR?SE@n~A+#1=y*yLCC&e*lGG3EJgZ41d#yzD4 z_kY2H1S2S>CQkonm_EMlVz9Flm3{D$Qn4kq6;dFB6m$&o0R+S5s0iuqsN<8f%JXlY z%gT&^duy8ew2z-bS}95+k70KrfpddfZ*(5-~Y*|1#HZS6A;VL$bj{QBKYeqo3!_^ z_0Qd)PuMfah!T%QT6yS){p)_fFK~^C+ddplt03r8-FUwVKUY!I z_rY`8jA03L{F;dvJ#ig3Wl4w7+2X{jU+Z)_dLX@z9lPNH`ql7VH+akpIAp164svfg zK}!22W>|)P-8bqU0g@F!Gp|b|L(;RQw0M`aSA^~ki77v^y6Ru%pv~^1ThS#Cxue_C zZdT2iI=-R|2SbMF9ScUq`Z-f0y6=+7@iT*miCZUW)8_hryMV3_JUBFG|HXpmRYq+f zT&((+??8PTMX#@7s#khiA$si}*In%r5F6Y%gjq#Zp-}ulbd;fnuwA4f#qA2`u*h|! za$^781e8$*}KS*-HJ8?<)H`^8gOp&Awf}V=^g3|ioIG$C zf3knreT4$xK(>C|zn4Y_M)f=$8Z-Yj!-c~K9;yD>qx;Y~W!zj9VR3_Qu8h+py?G%MAHbWwkSlEpNk9J)KP5w`GZzE9#``9 z$+S27L>_I%gWc?CrmJS0g0eV1!c6XYk?wu?EIYIUgM+2C^W31f z%vh@Cc%|3>D1*UX50m*Ph~AK?xpS@%2jIg}-!7ln??J*Yu)nuX4OGwBa}(HYugjWa z#hAIF>lMJmue84F_yQcWslF@v6p#4D2J>@1fG>v*(cXa_-gP@Btm};yV!YL5-#jQE zL+4$oY>aOf+HB^W{ptI%_7?8oqJsMGgBgqbcg9FB*GMYDj7_zsSMTo+WW=yD_Puz; zUloETUi_1Se|LP+MJt&=ZYeD4RgzQ!cx&Z)_(WTr)B=b>p@D@hVa+0i-KLZ8U3{DW zHJz_+Kr~{p^D)@b*DmS8aHq#9c5@i}2Tq&6H((4Y+MTcm8JrVzNKH?TLe)<9-eWtp zgA=8{A5Zu_D0<`v$8>bn1Bjrk7&bm<;ApHS;)7rju4H4q z>Het!o~_3I!0gW9Or2u4LTjOsv3{@?p})}X?|Vei@!>d0fHjj2cNh_DLQSF@o%?Hv zlqg}fprmY9s?g?oY1t7qnt3Ca`DTB~mV{wGp$J<0DSbT35L0w*uM#ER;Aazd{u=8k~xs#a7&y%@XumLkyhs+kELD(kggS&k4Rb%&Ng zgGWx-p|Ujk>HB9PU(A+W40F_j*D8CFbL-%`rrMt@Ef#I#U7*HoNt3eveZO!LgRMA8 zZpPF4Vl_38`a}Ahl{A6f^UpXgP(wxcgf-S)v+BN-it#BBg>%04BLGqYXqnizh~aXw z9$`(C8-^77)(dGrbKu6+2WfThZ%(7Qe@`Uc=^EYv*abA7@Jg}z-tYv8eLGPf5C}VO zF!hsa{=rF3MAd>^`;gSsfXt*Qq7Ro}!g*QKG3Y19zHgY<0g`8sh5KWUFC4YlMLe^= z{emlKRCgE1PnR8w982j8HS3^5V5&2Ww`k2SZHbY|y`DcxC5*HOk_w+tnoDl!R+J#6!IZ3>>ZPzfn!4wZ= zXeO3eE|d$r=8mfwZ`uj69CBWEP0w=>`m5wE1ehx zoId;+JiL%p!vt`9DCE;#(x1UKj_jwW#5Xp<5uNN`3!DA$RpVLM99|z+CsP`0I}yLA z;Bx4KZ~7sUrvB~9N77Qjpz`rE48d$StzPv@(vQm)T8Vac!&os#zss>~nDDhtl^**; zb;^n_;9`jW=sio|C`Bqvd?z0n*Iwb&bJ$B;{XVZ@g?~qJN@z-+x!eKRRF1{_tz;$- zoP)OZv7TyWm^`A|@jZ*wlx-8YW4vItrwJh|SKMe|WIqT|vL~Tak9Gv47jU!L&v?Kk)EO zNFeU2{pa!0)o5o?U;8DGUZ7ZgD}U){3;z=$xk`fmU1#0$i7lyGdVBZq6}PpmJIlL7 zBQM!0f9?`pal5)F_vcT-3wv&bw3(VotkK;}hU2#LV~@Nm{VwV2Pt!nqD3Lcxh&{!& z`KCT(P%OLs<)!NhEl*Utu5P7Er`gJwXfwpquO6!OveBTh6XVK8XG^*C-y=zXt9uEv zt3Wu|^UUMe$+yp};{7JL4_{5@Cp7PuT{v76HPKz_8_BexE-+*TGI$6ID zUk_gkQNK;U;rGcZnLhq}-9}6HRUg&v&#}=3c?|)`2D0A;6)NM53bL=jK2RPD&~Bd68G^v zL$RM{v8jUp&%osW7(a>e5X?+^>v%098dyk!Ix0STASeF?8`kVEqq!Q4ljpC8rZkR|MDxTicP;~3{)oBZq ze)p3W`U2Iw4tYwOC0fmLj5jMQ3H6&hg)ktF&u>J*RZ~Fp&pq?&`b8={Li5d&tUZN2r@0x!=D0=P)W7)P4kY?_SpU-uma3i+`CGxM3EJ%iQVMJLyMEY+*hN8mpsykE$ zSAbV?s8GiPy(0QiefqtNZcIT6W8XN}lGR*7c(?%>jZ|P>q-DTB%K{&>yzLI&>TF!Id0g9l$T1c98 z)@^kocKN%;WP0exfUnpmU-pUho>RemR~T3|-X${V-4%zzA3)ZG^T0k+0{=ZajANqz28paPlW z=o-?m1W~{|(`(QFLlnlUyJV!c{SKfIc$N4pt%I>rPK>H&mnkH`U=nX_5!}_ ziH8D*-$6dRl!i|q-%h9N(t5(tPlWSqQXfY5KoZSNJKC+5caZ#OB{iwlY#O=p97NAyqvz;=lHdD)}+cXHr%eKg{|39CNe;y*wXTCy~O!8KZem zN%bpyCWHX&*sh0ky^kqnE+u=|FI5v<8gH+KcY3pbKnk+#(b;kRpm87}23^@DWHPLw z%GShyaCw=Ecdgs!aXBdIbrx+QRaLW6E+xhDmfm-5Bc`Cp;B5g}?H}?oO|%3B8f+X# z!-y4ulUB7*53*jRFuZS4o!d$=8m+vU9j}hxfne7(;GPed$pWr@d&Ns)b-$R%K0IA( zP7^c3?Iv@N+T9h(*eQ&3??ivV(m0Cdnbl1?++aL0GACwT8M3$gE}TNKIFdYK1WIV$ ztXyISvR`S1$Fluq7#;z-PVO^w({fAw2#Vq?gh_|QNs>~b2FyZ#3y2MiJ(7JxTkMmX z39PQ9Pf*Gnl-&otvEbtYTqerABWXvDo$Z%LGH8MT&wE}9lHK!9*)^N_46^0lf!ef# zUZ%`|QR|wV9%OaAG8_naWlH97O+W}$_;^)$_j71T?q%A!qr}C${DWHp?={ccriHl^ zKsLQBoKVQ{5Fs@Qts@=LnKiN2`aN5P@(URUxboweE546F$wa&>eMP-Cubo15aP#t2{VR6OhRlkwZUaW*9(hE)O7Q85<+EUYNUjqjeLjLKu+D>q10+3AhsenDV#^KGwHM1i_uJ++OWH+` zD#9G)q`AU6{1hNlor+4FL5Dxo?LQvP**rgY_+X%x1ij?dpS&vtW}!!0u9h>9MA5IH z4q0zr?WokRfhLiQihK)Gbi%TF#)K^ws3I?^F;UDoaf-^{BBH!36PWXDWPo&!!P%OFO^QO=XE~dX=d+eVkSAQZiE*;>^xJ zRovV0G`_hTMWo3>*#pEk&Og9z=&J7%H} z$~11I*Mu^0`KH`^N}B3He9$`x$)Y-{8ihpP=xO>b#Me3*b_SZQIxay|4X9KaK(fov zz&b>9MV$YCQeTYqENn#BUiE_3J{k3&!-kG@tI81<>zGBiQ35mI3qbHh1(z;ubl+6R zh*;HMNc|c~xQb9RwhADpsIu`Hlpf{A$@rMU7@psd=O5hDKSMV<3O(5FcehnKN93AL z(|l{*nR}4qWhK-?e3e0}T5}gAbELOqD*&cRP1Yz50`0gQjnZ^-p5P$5v1& z{Sj;a^m43~knwXbN8!sA)k(w`g#W<({=xjKI;R*%)lm7DZ%->aUl8y8Sq$;Ekn$9} z^?-W(44D?!mya&YZtAg)_T0FGzRsv`)}@TXjjk%c2ol}(M`}KDT>7`J(WyW>!r8jGDe5)zOIL3NP{P;TeZ|QCF)~U_3 z;I)T(SF&lWhTnJ~_X*HVT#)$Rd!M}gG@+De(dg^+Yn)t$`Lc+9DJ$K@(lvuqM;nhn zs(8Z5pw=b*$?MLK1{dgKCM-X!`OR!ec@1-c4loU$t}0$^!T(gOjn7NHF+Z#`+j>=e z-uPb3M_Vl}u>@vWo8eMZ<5{#`)p{2c$nS|M(X6sl0r8KYX8hdy4d|QI-7yD@T|M#VgqKrah?~&QEN0E_~ks_mv zqC!YEDXUUOR<;nb_YB`sWSt^OB;!yv$2{hJy${v*_q)G)ANT(LxwpsZJUpV~^Lc;X z<29b6#6_G%W4#@}-AT55dFw5jPl=qs0T}KZ%Co)wD({0gjwHw}0NumgsBiEOI zGiRsx9eExm0;t8G90TN1zFIRb|d6_zF zflTDB5v>XY4FOX!#@FQcb{SVAL}b_+c~+N8X}q-7dd$j*wBn;p zGOTq@2pei^#cON1d5>0fL?}M7qGXjjq~837n*P`P=pMD$BhA zHzju|sY3@xRZWc0h;$}o;9D>UA_Q|l0v=ycFK%MpVhrZ+Dryh_e8EJ%4SBPQLRsd` zmN#p_c2r*Yb~Y!hbQ!dx{znu7ZgLLqFz7%lZI!o!PoV0aU$#n$c9+gTuLi>0#L|W; zv{A#Ij8p`Ypu|e@_dHv?$NX+bSL-*SC`r5i_W!zipcuMr>r%4%{DN*4{p^9)^ zu{1j$v>8pVJ;Jqw@=yd2EZ}$pn-g@gC}A+m?YgNmoEbr*CoRSjM8EGMGYm*hfmTp&C#`YbWkr7df|M{mEe<|8YTl*@q4qnv5Oz~{6t+sxz^y*X z(Dt{!3?OjNO1(IJ+g$N1EQ@Rgyz8lzw zYIq-IYnNeJwmnAmu-0BdY+M?Ea=zIl-kLZ{R0#MAdm!15r2H2Yi8r+S3U5H@^b~r$ zY9SoTrh2603#6c*;x)G;8wNm?l~4yGJ*?HR1<{8OMGEZ@!;@MZ0O5U5C{RK+jY3Bt zgy(m`VME}M>Ckin83X{iMD78^Kxs<@w0Mkb2a!x8QjG?+OpS&2sLgXKrZkpV;Y9aEO2d$8=Qf08hQ za$4}I-?i><5w4+BSrQXssKfGjD1?` zQ(6f5=R(!FfO<&aFM&(1Ci3TnMyQD#9=mTs%(t`62b9C#_mn$~fZ31x!Wr+8!o5k7 z@bvjnh}89>GZ$yPA!SGUj$WlFM8yy&@0tvg;L74XeT1D6NShsJjL8%z(&T6A_5r!O zS@Ax}Gp^->y>0LmyG{-OebV751UR13tW{Yj{U|=N=s3%XWz}5%i)i94MJxu73>Luj zmQU2$eqE~1;kC({;tkk9wG%e8c#Zu6Xq%Q>aeZ7%VW_;0tlFZ)%;rAKE7Kd9(}x}c zlIqhBejg?x`9vpj=7zSzn2ysi0yq=|JkcgO%&amr*9IDpN&+os18XUal;uEkTy~=X zmXxcbgs9?W)7g0tdceGh09{5USJ8Y+vRO%1z{@6f4DFA)Q;Ns^shn#^A*kAN>0P3gNJR$GYhr%<8||Kt6UtKw<&ujAr(iSWI95tthGw1A>VhfUhwSo4=<+5T0u?ecatI5 zcyqZv(1(;oMYjz4OchX?G&We%xUvJzMG}uX4wwQEs1qIb-}a19v3?r8X4)^5s>_Z)6?sy|8RX)-#da9|zM6AB zGr|z`O+rDiD5|q*w>AQ|XLOg|1aZ8(@_);TRhNLI_1vrRZeA651ExZmQai^`5)b@f zEQrph1+vH**#BsqU_EK2ri~(@jEYC^P)xTCMi8uoh^2aS&y?XB1yi9(f_!s?_~=E9 z4;}AyPNs{3V_+_`tElM*glZnWim1K}&=hFh*^cBJ^2oU9^=zMcOYMXe%e(~#l6{v& zbxSX?2aFb)CT-b`6{ldc=51k3)BISbsN!N zurZ#u0%+iV0o?{o5cfJ{RG~}R!Ybo@t`q7-Ke{_zf_u;@{Dw92A?gY^J|ieCv9LJMUFQDgU0{_!u4M|2EjIAB%K2VK9Y!bGO=O#H2O zuQCdY6pH~Z+M<2(hoarEIn+a>FyJ|^5!R7*mNgTiGgOegZ3sUnbhOHbU*v_H3LfdL zABEh+Lk>uDM(@BY{br;XSdwSLpnZn==920IwbFm zM`1AuCSGp@y8sGUj>H6xy$U9aZvihC1!M$;Orf|>N2r2|kRxN*BZ=l+*}55_aY3Z- z)eyFO{5YWtQ-wAz5PF-iHMz9+UvnN7y#5W&12lc3*BbSpBDNHXBGng@O+(3w0Py&p zvruC(&DMMC%^q>bQQZukA5$j26+k}Vf(jfC5`16lI;gdp0I;`_6+^b21g-fQ0CFeH_9x8ry-P`ku&IaCM`8h{AilyKB)8A2qn zSGi6QP1^Um{IIgcGCTDo2};-v_$VAO&3q1V&$X9xz(eYMjswBDNC1`n9HWYmb)BPf zJ_cxf&OMArg1y$pOvJ(ANm}0T130C5uaJvWv0ZOXWr-F6)S_nc@SWXs>oQ=<=-N>Z z%^Ni|#wp{+7*U3C6&>rXuj*6H^8@iWX0U(JtH@Z@SLX;DRgftSqH2+66x6L>BA)>q zU`|J8&ynX9QxBV%9$<=BJxXg3@3Ek-pLf~&kU^2^Zn{J1#9WZ8`=ILWz3^cS=m7OT zOoui=fyc@ntB=opkAUw2i|5i6gbSx&k~qgE&aT}rYoTg`K+;YdA6+5A!y-if7O$;u z0P)(OlK_GBJ;_L_uxVq{{IeVDYur6NV+dgjg>dpoUn1h&4aD?@Y_OZOU!qB}ATS_Y z1DM?mPbiq?Y6LXsXi6#{XK^^%&;Irw2@uQvG(ry|V2}JrV+Ow*>|_`=H@Iuf&!mZn zAh78GOlg30+uc{Wo7XG`dofRg00@DbXAnZX z1sl}3I#?<6^uhhHcX1>pOm-@*yVcl65G&xe(W(2e^M}U%3d>0U%j~(_?YkYnao8jG zw#AamO=`6@u%oVZA1^}o0Z0G7n5ngRbiuCZ4qnMwpkC`L!VDoH%$Lsy=Ru__l>Y@$2DM-i@Az5Yz|xgn6sqk4 znHO?Dn|8!mG>12$|31`f(TxVc7Y#t=M%jKrfUc>9Yd$$I4eVmhe?gG&{z5?y!esm) z#-H3AE2#q-k=#NCMZu8GG{O0S1qrM+i}5e{C%=|h3xWn;K3yK4Eo}4R9?Q48N~~jm z2B^UZ&bvU55IJ?98O%l^U^7q!#X>CDNk#${<1}Elt)KoYegw>bl>j<)c+;Wqhk)x^ zB;rOT0u2oYP+~&8O605h*8C~Y87v&U;EYq!08Gc?U?>43f^6YMwABNQyW+$E@o+wk-bM|6A6WY?=?i4~&3_TaAi6Eo`RAU;ld?D~upOd~-8l?SaT&RL4j$9E z{y8gye(`b$2IM}oh=-SE22-zL;>EAEGVz0Wr{<##VkoF;Wobf|9yu{KoCrl1ZRj^T zG}^L)n#k+=xRdQ@!G*%?_=#JEXY08+~;Yf$x3DDK^Cr04&-2IN~;Y4&l)2Uli5Ys7La@4E$TU~_aQ&dSb*rPE>ypjyBn=U%oh zsXp%UKvw)(OLXcYM%Vas9HpDR9JtrHj&U}(>yo?Ai7k}ZF=^}^_eL8lz#mrcc^N*S z$+#z#$4zt6o~n<^#eQyIHh>9LnK!q1jr}x)`hwW}%m0osQMk^&?dBKbM1}3Ln0>QH zAhtHv>ZT_?pY)G}33vi4!p{k45jMa%)+X)KE@Pv|Pp^YQ&GaeLBr3Z3K6>|0A1VTw zB(?MK%RamIrNAMnE|WlC>w==j@(oK^3ky}5H& z$^l6F{x*Y5n^Z zf-+W6w+=a9ulxHS(H9`onp`X6j!oGw!M*;-T|B1#01A+{UfxQ7P}NGuVjUs7Mulv` ztIlsTbC0RMobl%)gK1EXzA?H~=q?ru2;YI6%bc*Lw_a?dggOP~)x9?_>;3vAC=5A* zq>_S|3bA4C8j@(@f#jRRl== z6iKN4Ln-Q7w1fE;s{xG~qKs{cuDGewSZu*j3Eyc!-`O8!<=LKx|NZe?R8L?8+FQ(9 zZ~4tI{`vSnso(o9ca(blx*o1RQUo!s+jSa;?H9o>CEd6Dg~^9&;|fKSj!zW0wY&xW zS%fCUuZMYT^9g*JBR;(eRz^?3I}G8ltH!-kaV>mk=OQeZ-*<_4?sG0UEF?Y(+z}hOYw+#W$d~^ zgTHfo2Zq8-@z0$Ca%@X$is#y_)uvC&<`cNf3V^ibB>KPKdI}Zcfn>@0M&GX`}O<0*l1w}JLTQOA^dMf-}mmQAF9Y{x?N%F zJKK=LK5vd$A8cFx7+9wB2pfLZBwes_fjRQvdJOf2=a$C!C45DJ14gVof5kqo)I8r^ z+-_RGz-&>h9G@Rp)|NK>l#(=#yW5F5{F3YH%l2^o5U9_)Mcn-6qHE(J1InTAS8?mL zQUyb~$@bm*80?1BhazV%pF-qISwoJ!()(F8bUJ}!YEQaQObK&{wClCH^AZ#LT^2{a z7F-#-Ql5!XkIR8}9CpI9*kUc>4K9KA@W8rO&fUO?OQ%0u%N?>#Xbi}-ciqk~8}@KT z=k5HSCzmQm@_M>OICBhqiEyV93<|CevX@Vde80`yAxIb8>DC!~{@`A`D zxAQw>;-QMu|zq-uD4FoUc+dv=#fU7TMRxNFCvvj5Sck`U)S# z`qQqYbmtwFnkp}T)Y#YBVet{?(YBTvSoY5GVPm3Z$Iyy}TWbAuvD7!bxx>=>mnGUxm(A7igy4xYL@^R(kS0ezN{s+hRvx+3@3sO;0B>8v4^+dFQc$A=Fg(Sp0j4-W`S=laZ5l@_>zh6V*w zp>f)ct`r}c-jV`$Nv|}S6}?kEwDe(NEKVCItD{PFNFKKYtaSQvr>=OdvnOEir%7rG z6@OUIW3<~Qa@55g7ff)kD;exA92epa>eW(v(fpPDWN&wRyHR^q*llYaHJ+9p_BZb= z>WrN1tVZ13l<{+w%(#K2z_QORr%$JK3pCcZmo`mP9vi7TUc|@AY@5xtujZRDCy#mE zH)ksbm!*i^xc=ka=2(BP(c8yO^iHuaQB#wA?)Oh#`l*5Y{oYRCiUz# zwCo!ESIOY7t;W}^(XN++ldp1Odp^F5E!nZaR=fAh>}Y+A#cCvj-9)aukxgYmmJR(^ z|GRgRJS^oU*b>+JL~(DZ{=aqx2{m@;k{xu-F*-{Th3tj zZ8D21TUa1<>Tr)OSsCV5O}}TI$Ah~o)Qdy4VjF3b-;JauXG%7LBhIKdZga$VB~N;xXPh42t+}>pG&sfLwL!L2S^r%=p3+&9 ztq*_qhf+}gjZd@Y2~XTg%#KRd*ocj+4>0P_-8j{gsPgjkVBy2jA2ji2$()7z@jgH3 z7eBhL8mxrXH=t8)x(i`aAZFy~2cz(vpuZ?(LN)bDrv3yYEET zYeWXX3-0rc{&g9P2YRnLfA4?ge|M3&*Uf2OKdCoJ<#rE13Di;xi04-JO=<0{lAmF* zh7QtdIVb1jtM%ek_h&kDD+i`5c2)(=7{7BnIAy!Ds&r<(InFE!vj15*5|aXcK+P`lbrDOlkz9kPIwC*t5|Cw13{`yYJ_ zq5ayF&K_;3@yq#b^T5@oy2ZqS;SfqjRi@vsf2Oh#;top$DL?th>hC{U0BxOTLL5B5 p!$bHY0BKhpDT<2tKls2^`C04H$>=>MHrwEjnzE)+&WX$S{sWpn$i@Hw literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png new file mode 100644 index 0000000000000000000000000000000000000000..b28e44a636b0d6d61f9cab9bc50b9e103e38a115 GIT binary patch literal 49246 zcmaI81z40@)HXbHNehakAl;yJODZAVAl=>4ppp_IB`wm8bR!@!lp@_RfFMYB{Ch;t zIsf;*-^VrM%z0)%``NkHecx;C6ZT9|1{?DpCIkY(mXnoKfj|(ez&jir6_LM- zZi!hve*8@C@nfoI&JN~Qwq_8BY*@S|nwDxWak}o?7%@u(*upUi_ksZoR;LgG-9pvxa}y^OBkRK<|b#7Nf>&~SJSJ0rs}(aLXC`?LQl zMUtE{s7SQF{B+$)j7=e$fdwHzaS~w9NI}~bhwNCF%IZNN%6%yctax&q#bQryA-(j> zx^2Xj@`PL_c;6 zDPdB+$BL%JW5^IZie!(BR$uqQBRc8s>0Y|6Z2#^v1utw3^{JO?sQw<736aU}OT;=V z*qP_+d@3eI3VMTg0~l}Kb_cT~Ee4<9(-BO z{;~dwQ*=*jBxHZ!-uCIF=w`^z{pX}dpJ;3F1Ow9A6`!w_(`eCTnv`e}js=p(FD;H9 zYSlEnZtCh;>J1bK@;@PcQTa`NRd9^|NWd@lm}nT$By6{8dfA+oLPd*kCNmXjVxUw@no4qPKaevP%aw|C6f4{Jb<>N~dPFVa4lw<(2i zK`c`037oQ0J96$I`aI2VPs>G%3?M0gz3+r*--dM(D29hgElwtjN@#ST?U)wPuEs&w zkCc;1#)4GTCQy!Viy+u$UrzZjh-Lwq51TFE>B7A_Buyn;PlU5}!5tEDJOT+R%TNmA zNG&S&P{Pk*J~3hs$il?m#y)&Nh)+#mO2>Xz;uYd#U zvREaC*M*rn`S-2EqNGU}2(ce84`W!;381h_JRPQ8zgLAi`-&%%ei&>0Q5C6gAkK4$ z>K9kgJPMQNEGz_1+qpP6c}ST%m@4iSQ`iI!onnr?IW(elqE_tqQ75d|e&17Quq$uj z({9&^fDhVSK<%>T4w|Rv)f+L|yLdm*(~)`ZhsDSxJz=0#BdEd0!B=c={{Glg<^h3Q zsD`9R-JQ6w$xf2auFk_w;m+kwwk4V+NSNd|n#tHEx!4Cd1p#VLbe>L0S<~v%9ma?- zFfx-c^E09_<}hHyhsQI;yEBoiHWVMI{ovM4vlTyjAoJF1<;P0SO34cGo|H50DYsOx zvD9vEr*b9ZK7$Rz;XBoLjqiB-`ucMFDEhwkA;RkWY~CF+WvKgQUMNecc4@rId0R-W zto@-iLB0{VjE0!lTjLYzWd_4Ez0QKnJUQD`CiXpc3q zwS0A$wUl+dHQIN!!oz~NQPxrI4YI?@4fTzI4c_l)4=wMRJfwcK_mJ4;=XbumUHQ0= zRs|&ay(27NXFhnT?Bio+Cx6v?$Yv*E$GmmBL?$BNET?O%Z;WYts${faw5+6H>qAl}AE9&+8{xlK~~NXTl!;-krm#XGyIZ|h0QWt;;O-CEC^zSWJM@av}xeq8%DK0N*7 zk`9&bJ{>{a09_hghC+$_vOg2-|(v)cK3@ZZ*zv&>`_VS)e zEjy<#OU*g)W*=aPP-&=zMuUcJ5p|J3QRA4Jjo~Ka*rPE`yHK86DP$>!Jc+i-x|-80 z(@fJ5?)hmcX>In+`r{0r8KRXxD?iG%pD3E(-oo12{7K2T&i7LAw%|j7CNet_0kea> zuNKn&F0O9Rn(R7_2a8w+UdpG#mfjHa}Q`S|t_V^QuwL`UcJUhD>jdTH=e zOHcX2-EZk4fuQ4UJ#Dlf##K(KTD6+q3zNX{pf^~%uYz8sMWo&-(LX4O)O2r|sc5Ra zfEsTv?YBmqx($>CKjlhrWpmSQFd}hEcAPwfwLm@8EY)tS3+Zs$OLaUs>#K}+&{WWI zEB%(^ylmZP-Mg{gKvD_-WY;&nrHF>8e;-1f1a|YbqlgAb}1eaJQ`vd(swWB z^$+aJH`;Gz6?zr)2eJn2SPfW1a0JQb{5n@;N0qm-pGGT3#gmcRvRae#u?ft1Q15qU z*T~sU)1E_*t&|FB6{T7=tk*_x0AVS^s_@Sx-Qy4Xwd&6CvOGgP!6WL zoL4=Rnoj-$d!zZl=VCwo&WAf0=TEM`pL{yYIfMrLpI(lB&G>C--+GW-QE7YP9-H#rAvvk%GIKQ@dO8>`E8t4Y@BdD~I>J0!msGQ4ma$z|Yo6-OlLXC8llC!_o#l7ZPC^1b8wcwOYg!v!gzS$A<4 zYojO!5l;$*Y7i-NaInK|y_il=^9dGyIuehRkHXMXwCmiSp;Xb%OBbwXRe?CVFKULA;<{#ASmD)0(grckpA;s8i4_V^!qv@1QKEeLH<)l z5q!e`V!#`I&&?;&+t(1lQy>K3?U{-Aw=`l^CelCO!@+M5an;9ia^O?d#M#Wu-o?_v zRlBG(4P3x*l+|{DK!|DKZv;7&2RoqtF)KAKS1kp30TTy1HX~C9V>32SJ4bju5Ft+i z@YT-D)riW|&eq;Vz*Cs|cL@RT9e$aen(B8ER~un!Ern-Pj~$%NsCd{o*f^*~FsZ1h zgq%&y1ym%Z|J)A#OPJcy)zwjeo!!I3gUy4R&B57%os*xRpPhq?or{YVlwfu7vUfG| zWVLspxvAuDJ(6ZFCeBulu2v5ARPcI@j2+xug{i6G5Blftrk!SriE zKfnB+cdEOXIX`x=1CMkS`FFbh-2C4k|J*3V4)6JYM&f3ee_sXTEP^S-{?D9=VCKX` z!WSc%m86my_ymg#{)Zq1{$aTJgnw6^p3VPQ0)dD@3e+FZgTVH>6)bL!-Jh!n>F|D?5WWL0^A-5kNZ4rWick@DzY-4Y|kgypq zRj?_u9xYUB^}DQcIe{Qj;oS=4LZ5mr_JQb6(Pzk1mP8m$L1KL1OUL3ZW-3q$f`~y4 z5l_i%e_a$y1ra9=GZLd0gCK*KEx2woV);3d4=*6|O$S3(^iBB+aGkVhQNn^1l(%W8 z`i%0s{ypF)EHr7(IB-2_pn@Q;j$%h5epk z3Mudc6xiQySO3$@9R*0x7%!I^B=9Y|*l@y2+<5Wd?GeiqLO`F|lXpZ#b|nbN9F@Cc z7Yyo$pg(vELAIj~Ay*`Q02+iRB7bvvLJrh3==4hSD=ld7Y=!vHO}~tvK!SL3kwWad z6Tp48Avok{zXvmv2~?jSb?eNG83A2KSBBUx@OSlhX+W*Fgm2Nm1H~oKDC}>lUk8_q zqFHrN18qV3KPk8K<3R$o2v5%EueXn`>(|obemO;<-}-D)?!KinRb@peywir{eC76Q zE{#c}SXRAAgSKA|5s8yoeh7(IR%Z`2XzXp^OHwd6x)|k-s=N^;7P~mepJ@V!rgpSC z-g+SNbW5Y_{Mv`mg5fZhXj{!BkGn7H!+YP7P!xq4qiuBpU7WO+F+}mE)t$+pwW%_= zzR@(GTYORX&Thdcn+neg5w9wig2HuB0(>FHVbVhk{7A_6t7(6I(jr4a0QJNn27+87 ze!`I!7cfwPAH=g(zKjT_Q4eDxm7aK8woapM|5d9t3~D*k=r^#G9$< zhu;gO9E+0yQ{-iW6D{8M0+C1b>cKwQ zvm=X~EFewQIc4isIm7zCXNn_Pj}>dnSuraUR26BId>9fr??2=`RCt9+wBAD#EiVk) zyc`m}mLJ!1!Jc*O!2X`AKx*iQMRYZPeI@fa1Sj5*vS~R=(3~}0do?e^?*qTf>hM;NS89c7VcRX|+f^=pB9aQjd+xqq8gPV|vUYD!LA%WdAD8A9H@Svk9o3b1vT%loZQg)PVI?MW@&9RbfCiOzkAAT!+@~M2WsJHG? zosX6oids+C*k$@w!RHHAKz;b`&BjedH^tN$J-%b;6MJ>}%YVf8cvfcGzLB+Jm~TBZ z*z)Y)$H8V$5#5yA>GnPJd$2m6*pXPb>>FXUmO(n?yXe^^yl2*Rw+U zG|eIm{V#T9-gB8@+D%o39ZgzLT2EG*ZJ6oQJJarFjB6HaiKCG@LLOOtd$1)CQODkL z5@bGvx9L(k_t7;*rqjtYCL|;Re^1Ic6I)%Krv0AlDvAOM4-WTlI!eZf5>Wgsq6a( zDXht;hA(AJ0#~5W$XnW{`Cj|0okiUpud$y#b4OH$Aro9p zGWlk$l*Do0Q8~rD?E@~@g4lF=@RcmF0Gg6#k?w`cCzeSSs28MhIr*X|=Dv!SQ#!-i z*cDh-T;y5)*HHJeP^}WFjMBRYy<+n>ck2IeQdHLRp#w)S^2NdW zeP7e|LFTvp7SX~-nXjRHavvRT?V<4Um2Ue@^u;y~w|SAQ$de1{zaSL7tqHgEWDj|A;SoI-!$YS8U!jc(~( zQED8xQz;K`1NPV^Uer;VGVm#T^LKn>&Vc|t^(fDt-bEW@w^4lKp+$aL$ru_6Diz66UP;S&F|Mmi~{@yn5F8_ik=W0cAI zg&-`l``asS6=%#)7gA#w|CUVg>prgiwV_0_i5h{<@Cu=BJ1l%=C6|ns49;_Y=XR&s z6-J_??MLiQI}#QL8G7Tz+RLunGn4pMWZ(SRj(_B3^wqp7j~`9<*^`?r)oNBB=-KPn zPuo|KzMy8dE0At*{Xwmb#mS;nK?$~X#Xhi3`P9zEuaZ2vl`m8l_53dn2Z=QG784jB z!dL9v&IH!(##QJ;q!53Vp`5{JYWX^K+o(4eDYSnE7MQ>3!octVZWbMFy>PVbm1r$m z*Um9%m1p5;2&TC4mMH>mh0`X)6?)r-*Jp{|QS*_Mq6V1N(IRKEm^#E#OMxt>=_ss} z#wcT6hohP~5Z|lQHF9G80*T~T979I5NA@kJ<{>C|GBnY!xWPDyI#X(iDW>oY2zN{9 zRTm(aXz{yuGFeSUTdQK|*gq!M%jha1__85%yq5009ZUJCViAkwXIaArllJS&=l*q+ z8w)jE8+9F5`|W03*0l{RhVkp_AEuus+XnHtMA|nWYMeerA;m9(RXQY_rF$-gJrviq zy!B0Bd-*XBXHnrMUN!JOAIb6``8ENAto1m6^5^1KsAGWGeZ*Afw)!&vQ>dY-@|CLa zwVY4u<&n!i%V@@^YHjY4X!CwX&#U2B{-QpdYpX`@Y{tvUIzS>;%e5On ze-h^dNkQSm@2U^oe@>(_iZwM8pZB^e3jBD`t3c8>^A-xmgAq-J+)YiVF;)8W59Cy~ z&jonCX9uy3VmnDikBd>`v@kqWaWLNm7Ot&5;vNt~*GguKokCsv?#%CoUQskhd}!@v ztJaFKKm$hob|&z$!)5-B?3q0D&} zL+N`0iAtBs{&bo!uN;-o$I0K?iW)BRF26rsc`dNi&Q^~IWiGOo(k>a^3%4kxnZ{&I zgvHM%8oXdEdEY~sP&DboMM^3%RMgOf2<-+Aifx3ca~t34s-n#LFkY2Fa;}J_?5bBU z>rmwj6tko!F+Or43U9^vRYH9ZMKW{MhE5jeGgwp8o*~jprg+h&dnuc;`UNlnzDh!| zK5=(8J}dyDOb~szyAco#&~Gzac420?e3iPxzZ97Bkeg+Nk6uIQK6@8wggUV?%bZ#t zLLQf+up$YbA%q$w-V>dvggpLJpFe>`7QH*WW0{NKMV^VG!@}d@Ab~h~CHFFybhb60 z!j@O}ExWdnOMU%1K4^xq4<5p-6gGlGBPIX5IBQKqj=Jw3f}02(^SE z!Cq4@s6uM;S#GN#f_`!#q_kqw_suL%@D64vTpGW!6EiiQubZ_hb1|`di6UF zQeE}wP2)^!HGA#nDJ{U`IFmBXZG<0-Z%tHW&ydEKVYM^XCNKFa)epSJChgm7qGY0+ zA$FkbnwGIex0m=vk8s0e(`D0b7^!ySIm(aBG2d` z;+v$&Uolfl+r0;zOKr2I-^jv#-rIVIy8N3;U=yM$0Arp+3kVF)Q{u5^@UX)tn6S?e zp1at-%*1w~YbOT$hXL8vZ|DB)01g!{K&am0e?Y)S{=KR0(cOBF{F_0_@_=7VqeYu= zGXVKI={Leh78Bz}m?PW=j(N-$WguA09+0m3$%SzK8#?E*kC_1I+(Z0wekFNAu7{)? zpoW0J5A|Z`6J(_KEs>I`Es>GMsGcC(;!G(oMYgPf-lU{pmnPBtZrmf@vgah=A%qB5 zJk(cxN@P^e!vT9^+$&FszF}|kJ0q_KXYjW0B?L28qh0L0YFW~vtvhc(4Ix)_dIsiF9Uif z@~4f^b!K0F6#+xEjx3LAM_k2SWS~5_=7*ag2ur^^JnEZrh`nLorGm+9PiuJTld9~o zc{ktV)DyPce;B8CJgJC_k6iLzs=XPIk8T21Kwl+awt<~ynPOMN`Hhsg>QfxtdjMPw@cZjd+E89@5 z&*fk@$^P4(kC~51NTJu1BtJ=jB;xhq5#Z`C31fmZiia!HVI9yX@4kJ~wYD9tHP~=Y zvpw7oVPU4+CLP`Xa$ue+g|NmFqtN9?!}h)i5Q%INt|Xj{S9$HjTqe9}{N6YyKrsx8>y_q=sP)mQrTGBABV-aYiiu_**n#> z^jje{j(G9x%#Kv9Y`@4PJByP!VV}B|j7~aPT(!kZM z!bGHc9uv@;Ilx?%OL4*xN|p<16e8dtp^^V09qk#;hAOW6P(TgrGa%E%0cO{vE#)RO z%ns(U1X=zIdR&DQGlK9hD$7GhsUoobF=(Ifna4j{aU?}HmV$6f3^|GqZ!Csuz}rbs zxBekwy_Y`Y;Wwn2;>hPA%6OsJ2yvXBO>Msmi{Vu@U}XD(dWSKR`U$zcK4FqSyahJO z3Egup>7>XvE#jxez@iY|4F~*RG6S@kymP>MW5$YLuVr9Ll9J~IIwI9Vgcu&@=fw7R zpXp3Yp72&Ry~-R7B6U)C_DDo<*djUF^?Y`_4mt^ykVjL?+%IN|r4>9=rIlu;l;VwtcatJq z!efMicA>5M#sZI^fQ(mCg&j`4L+j%B#b^KPkjQv?u|&d-`O-*%++PPy0IC1 z8O6wplWf*U)P@0`(nGj&kH(eHn2Tp)hrU_?eM;?~CoY&!6ofeO>CFKq^>TAvH-+ZG zU_?%vuJ11isl*ISUgD_HnPlVmt4zPlRLBWD2K~~&)2g*6S1(ZM3S~F2*C;6ZRR7Yt zFmr&V<1lX^O_AlL6A=44BLWidWmw*UI`4K)%WXl1NiM=GdDWVHb-dS~P9@b=|c!ah-IRFGL7pOt0 z?m#>d_*veOZ`ASH%C`#*PnLFpR{CQ?x3>cj`I?<17(sXM;&YrX1Sh7bvsJtgPlB3; zYqykka=)V8>_3*1Z21$nvgK-}7zP#9XIRdUumT|}B zftZnR*y{Ja(Zk_bFUBE(-B1z8xLl+8%E15;|D;NBpFUenarh|f@bv=rB1N{!}`bMWIV4s~v2pXE)m=VZe>G zTBKF!GpEy%q|D4&s!p{v|MkV1C~Rmr>v0-Q6yIk?X6882vev5?`3&h^t0Y#n?psqh z$FQ741826@3ulY-t=dH2%kkE$oE%X@pWRost#Wa6I#X-XA-I%mcA;f*qvkZd*muJ5 z1tvke1TL3qv2)fE z0;*YKlopW!b9aVRR87@6&@OZI8xO~HVW(pyx_N19xW=^OB%*#zwwkt%8ScJ75c+o1 z{b>pW4U*3?W0e0GRdN86yZqI#q zayV{iM(zi59?4a30YB1f7dUfU47vjOi^lHau*^Rm#yzcUX6Kd0VH9+rIgsV!7dGk< z0^PN#kjc8MY};tCksSHP;7s!ga6KyPS!}>jKENo}ZyeKVZJH^5N43Sd!8Csy z++7u1CFn>99Fo#@{5)9}0K;M>ebFh8rgUdc&%LQuyphZgQ1qJrD`4&v41^YZ*yQ=uo`*nm@G z9EeBDO=lZL+lOGn^bIzLvw}a38+g~lB+r8O!D8)mE;v*dEVy(f{j`o9p$fARQHGAI z)z@L_fX#aRE--Uc;qDeNnh(MQ?ubJr5~S=ngKFM2IFX0?1Zk`kK8bCYhyfHq+w-CI z_a1(LfpZ*JMyGs_CS?*}@!j6Pn#U)>CW5U|2*2oQcrxrfn;k7ORSzVuG@$9IB@{f% z3eqcC2?G2%VGV3>E^YRC(G*?>ca%a;OZo7@bYbr&%FRH*5^!1Vj~$KAG@AyRY5vdo z)3prtEuh8rE{*_iQTnI?-57Z~adl~zx znw6f_iFW09u;8^j{w=4gDYG@s&C2{`EC!RW1=L8N+-4SsQu~Y+suw2d^_uBgqo`LW zCO?K6Cqu(#sb(7V8(d@8&W^U2hZx1Z0bQvx(|YD2Y=EIrfFjqQWfs^+_FbtkA6uoL z%afh+GkIpfJlOsSjVh)5=u^(Jj)(vzZX?UDhIut>7gQnn9lpg%dG8H%d+W*|UeqbQ z!Ms*^v@r@@jDIHbE^`Z*u6xv1{f8$?*DQXcxDlEn?CC!}P)a5+SxVT-R-KdM6b0nc zHH&JR1oYND0q!@Q1!gLT2&@hZn3ut$)yEi9{WH^XN+Q|`%X>bigjFp4_5LjH>&yCT zqsRsO_BXE4Xe}GvHtD8SuGPv6^b>UJ9J7StCB>mp7KGK}5!yA&kE9(ZGjSi?39{$K z+yXw-==LxgF!y2@Nw=m995j*vq$M(j7~cVfg^Es?@#sXq7<2+Hgj<-Ybb<7|)%F_P;Yr&+i0-b$7RRa`%+ zh-u5=%s?Ui@JtxmOy%jvL2|e6%HM#y-#|C@&tSABdh7?Y5hpV+k;JRvn&u!O zxkoA!wV`&5OA39u=Z6bD(8y6U`v?@#wJGznT#6ULnF^#qsg#vrQ<L2N02aRzv-G)=6z|Y7R06R#rMvUo+8mJIr9&`B72H{mP*@^iyYU%i~I5>4N zfARssKDo!t*Tt>YlZCBnzEQC-BN6%*k?+dNwLGI6R+0$=FN6=lNgs_WQ+>Z-25&+?^y$R}Nu?0dVf|G`(8*-_)NAdMWk9UMG4 zVXZdj283&5M)Tyn`Ma^-Id?XBkxLAX8Gy^e^FpMf^kFzM@A}N+@+jd|iB8TNA=fwL zw5lRz;wi)wvoSwfEztKg&# zr2yMb{6N%@6tC)ZeF2a?F%~#-#ac4&Vy=+We>1?d2!?uw&3_Wn;Z+xP!YfKq$u3;| zf{?6tnP%-h)bBjR7MvAQc--ie()CJhezjh=z=iHU2aaWDpbUy$u zKU=2WEHAwTIS*+u^#-FeVGd7KaX&dll?oX6>`H}bX4bP`(6lsY_!Yfx*aiX*^izyF zZp7{&0bdXeo@69L4KG+66~0G0Wsz9nVbMr%ZLs)0l}4%T&pfM6%DDLv`_9F?_lR(9 zV<$)gBo*1zi?zNUimkMMLqH=FqvlQM%s7&>HIV}1Drs0r)eP6PMfU&;PJm<1Z0d&aW!7vTjYVv=&)O&w@2(L<l4=!M!6%dF12m_#KJv!i4unodNIK2sh7{5%f0KxVSq7rb+-3N|8 zauUDw#L~PufWW(AbQ{_Dahqu*FiUQ@c*rI8pDihY-FJnv1^KyR2{i0rP-P27QY1b= z`wq`Un8k+AkY6vs+fDzc-TtP-kAWfk!@u_a!@r7ks>5q$-KU4QD#hc#<8V3{&;tV4 zZ#uY+rGgJs(ujc9OrSQZn23Ph6!lV(MIkUyuINm*p{I>LHinZPYz;3$3C2KpA- z?r>aQ9?y?VRol#}02G8hTB@(>$E^jT!ZZ7HbNv~j85$r2S)yCZV?@F8_`#p)#(zA# zK^J)hSeXQ9wNn4e0F&!5CpVw#`m-}wqG8|R3t%Zf9&JslLK9e_`dFLD{_6X{rCGHF zAgF0*5GQC)aEkpIrtH^$anMUJy~8|kg6Z0St1`1ht$^Tg{p;hS)P0QBt23MT97c${ z=fDO?mrlHQ+>92z%6w!q_H4GnO(7{{0PoM3+IRkEz><7++W)jIAK7jwU3lch)owTC z2*5{@7O76Cm{LLj3~g{*&rPHy20a@qQd)!PONVzgmo~x=bF8tE+8nIeeaQVPey@iUIT2Z#_lP7 zhEL^%kja-@FP6gS*T1VOx)y zmp=eti)St_YdK>YNEc3dakxRNsTrSHV$dv*1|ZG%Vr_0X{j&t7ziQs^9RC_4DcUy3 zeQ%|=YJX601jvbOC>#-LBHdeLo(qAMa{vi5)rJeMrMj5naFnfx9M8Bozak0JpbnoA z{cYj$y{V`y-3X=6!CuKCl`^Ko!`k8ZjL@9V%ntvlvX3W7Vn3 z1Yz=C(Y15@swU?bz`4au(Jq1r?%Sm*kQ`A3Kxyxh*Q+CV$ePD)s;<-1Em0MZ&ml2d zFQ~?0zNHH6+Fq`idGTP>ulGa9s7GkN_w6}#lgq$i1WU(tzKGaxgDmyomp`Th@xP`c zxCShT^-L4<7!&RG0j4 zX#HE&X^=tqYtiup5d1PkW=lfdv*j{LysFRTabou z{}<*?Y*HTAb6oMw51}ihe@xije|DrKJ}|dWZm(0Pst577B?i~d`=!izt&wj`RTuR; zz|7+_)%mOY)M`RL%a;y2vw;Avga0`xx$CfaVG7qPMCK~CsGer=$YJgWcvUazKEN}i zf{{-c9sRVvbOxrxV{FXXniEzvJsMkR4~8M>0fP4w`y{IrMu)=s%h3_os$n=?ba5qmr6hX!&cJgKkoC6*;a`0m$ zXXg!oi7or~7yzVY0hHd0Hto5|K35%h+$x2`N*F$8^k>Ks|1;!3Df(l%lAaFlCDP8g zjq86)0pUA??cF{>lnannm9UhcX{TFj&nmS9(0PV0 zMSURCAx^Z3Rx214`@_VnAN9Jyeg5vEI!~c4>10?jx(BI3b;!7+h1?T z{tp}7p^X>H2=5jW+P~desD!d*%TRcA%HleH;tC>{7N_F55oMK+AwYNiAFh)FE|I3( zH1!M>_6cl%rRNMd0Dt=;(*5542QsL_fI0k`MH<|H7i3#sJo6hnU4YDp1T61@GCK-4 z9>q6!Pa7dfs~HVJbR<%oJsS&fuzXL-&8}%iqPGK>BmORkRr}`h$(4G8K>* zW(hKPT>BXFhUMtfBz@11s%O0G@&L5?J*i6tQ?D}BAMC)zav)g$R>8Ho1Mr=IuOnBe)}G!^q=WapM9{4z5&Tb0WN_OD)I#a4KQBA6!_Nz;7GoKHKbH+ zGoA~$bMif2`&5nBYxst1o8Lh1h^G@XAIr|7{UExIPnmhMY z0oHzxs(3N<>V`;AQ*{P#`mYYA+4S?=LX&yk`<8YPC_zh+d>sb0G4v?xgRpTv)BR4I9H#RQ2SOrZtzsXw9gg6&7c|h%yf-10|g*Zm+9%l_Ccmz zT+yed2Bw$HqXnuV6h1rR>aV1uDOfdw0Q1oWE0w@)dnWm%V=hR_8)w}I`mZuTFJYS2 zPMv@D)dt0*|HTC8+Qv{qrr`(#IL|EZLB5W|1E5gH@K?ig2myYpUmGQ*@F#A96Kq64 z`IgfNHh>U0f_=4@*g6bQoZ4>HIA~_H7Dt4-UUGQALC`~?F`qyEM?(M4Mhz!o&Nm-; zpEYe33_PhcSyqC%^`vdi&1vk(PsRPq+CGUyt188(nDHAB+8uabl5yVH*LJ?k;rJwV-NC?cm9(7xL@?Me#(MZW;ZgFcutkr-6c zmiL5ar(ZSk=6nH0hR+_VFWq;#u@ zef5j5C3Fc@9uB1eXzxX92Fa3&@H85Iw^4P#S`tQQfM9NlvjwKhY>>sFtIK;5vwb7M zAEbT2#tb8fS>NJDR9lTmuWoEtjVM+|P8bEgINoNh-CP>g_PZNdi@WUWLzxBhy0hQl1kV5043>5Zgp2 zw%osN1s=}(>sEGvZ6oo{{PG(={9j!Q1o8AYv)#N?3J);{!x6mGe-M0|F7>WKIvvhT zvHY_T1-ww)zxGbzAA9#-2Mk8?{~i&v1J3C(RvZ@)#^13?<>Q|JGRVDlwEwq1hM=#| zz~h%?hVY0V9yTy@FTMZ=@;6Z026FVoafJ_JVks8P&5#8ngT!qA`fP-tzuN%LFMm6$ z^-Nao6FBV!;fn&0c%I;e>hVGH4-bn*_y^Vi+WrT7vvSlq!=Au#|BtRah99j)a=#An z&a<@6)!ZI$Ion`<0dh~nG&`6yOZA?p=G)x=J67-hGgd5yEveSyrQ{9zJeF`7Wig3V z7$gRNL<4fyy*Fh1TzKMw=KXgY7GIODL)lwD0tdpg`f~U(^N-^lCgYwC&K!6UGz91O z&kzy7m=QFjO%!JM>|u~QwjroxNuW#tm{_2cAp+KDUXTzxRT!QSWYxG;lMVND_a0hO zZ3=-*w@$m3-7loBTh-$#z_u2E0NvgcfP-wC{i*zMU{lcyffXn`Fb1+}#2i{q=>QXZ z4<{asp{*xNSOQ?D%{s%=$U*4qD|nVwqP{!adIob%J7Fy=to+s+2>zot0DF*wM(Hzr zgNT|o3*Z+ef5#vJ z(Q<}eEk&^B!qdtF(I{lE0SY#q1;kbr_#m@u(aQr>#$1mXCz&aeXkk-KWUNu35DWwO z&j8)QoCfJdQh<8X80h>utJcm@J~fBu#~6XEjD+7#5@73KVIo@{?cz02CY zPuLw@gm@Ao_(f?^vVuL+Jpk4`fC}&TbKBSJ+Shkkef{u?=4GDHF)-UuE|Aq7qPq%M zh1=Dy>#HIBqN2T2&y#+9=8i$(AVe4SR?x3Ms&8(!-JSjj{)Cy zs;XDJ(Y>U@ZYNe@xDN}sWev?VkigB)qBtb9FYlG`iVc~DL;1otcLAG{`m%!oEM-ev@{7pZI!FV-f2MsnZw)3@|qQTlsM z69m~y=9~Pm=KwiA&@i(Csz?r7Qg&m)ss^BH?WdE zCiu+)A%W!>p_if^JU^zmPuC=C2|brn0>n|Z8LCh+ zuo$8|sbth+%a)8dd0_z~7)LWMN-{qqV>DL3kUF@6p4PepD(&*(=x4HqIatf~4UeZ# zd_Z2}Ukcrm^(R>PUn9n~u09;T)v$u}9OU>n)3j*!CqJ@DZ~>B}fU!5*u@PHET0(ya z95|3wU`Kh$I6KtBw&0KTf%;?<0QrmH;MHa!@_K`LIF%+c> zv(yJ&{v&%27XUs^S4Xw2{2d{<9@Fh9mj7V8pm#_L%=4DS$YO0|2yyeTgGK4zlOC(E zM5L6gkxli+(W}n4_`)>w;75Dz^U0xW)W0D<4xA>S*4!q7>U!)FJE!Wj@mF178#iI! z`K6#+q`{h8*;as@>JRm*xp;L8H1%E>+JG$j^Ci|M5U(Tkm}V2XZbQNyhRZw0g)nye z1z*Qsb;~$~DmhOW(Vy`?__I7n+5c<9DSd$I07S*l*ri$(L0ZGB1aF0KwH~v=T~C4AU&L^{C5=oCFuWpG9x@NUA#Bz>HxdX0tHj` zDM|7N-&Y3uUbdEF08$$Lhm>Zov_3>G@i)&$fafvrW56}>pSRgSwuR(tIC22u(Lb8_ zaI?2Zy=Gn*8@z)8wn1V(e|<6j|K^MTUo8Upb1mAx5BCB73R3Le|EjF90m?Z~-j(g$ zf7OyefGYdSZeh=I+@gEi2F|KH*#q{hfp%2J3~%Sj@1tOwZp4{ne-i&p??2lpY$_OS zU@2X=jo$g6V{biDZL4UM&Zm+4tLVTACxP=js%mJ3>P6BZVS>4kjVUm|@Fr8S^%DXN zz!_?#wpF2uo(`qwXs689Uj9SIQ37ro&j3^lF9;WalltFBx{h7Nz^TB%e8O^mOPMrZ zmGfF4eCo9NHM1YX1z{kX9jK$d63@K-yYeX&bJPt0s3hM84B(1L67UJocKrkbUyl=+ z(nIinw#WxEV&?~9H1wCCx$t9dV=O`BfMJ1G5k|3YZIl*=?MxkKSw=jQ`rAfFwE`7d z2Yx-cG@uBM017g{17AZi0zuxuFhD_+adRJ1cxdI zT3aw%Rs$fcP43SxOA~}^1jZfI^jbLOIWEW!6@;gIS9Jsjih%=_s6qH|kmYa2amQ5L z1uXAXj0v);fe|U(CeV^Y2@JFU(f^1$?72W&u&PBu0+ANb7&b0G2QXXk<9jRpL3THj z!Hgj`tixaI$n+T`OVJ4Nt;2(nPVl_hEI#UdB6xOed00pKotqWK&J8hxODbSag(|eL zC4&U9#8Vd9qsuqpX}MGAN6QI%#Ivj6bX4p?-6y;%42lc%MtgYL?wg&?$8hNfyq3U0P@eSV*9P5*pyfW@IKn_6>JiTb zM-5!(eUD*v;UiR;AY-@0?@ReY#!aWLeZ;WA0uBWOsl>9h{Y^v#$_GWPe+JmK6p zzVNvS0~6i&fFqym9_ZWfvyK?to4!#JB1nPVvLwC$Waq52!_DsOPwP4Hpa&&nV-tdA zzgu{D@Dd3QZ6S!t;4CR=HNLpNEjS}<~k5Ko(k-;=P_t-9@c?YZ>H~?}5S<6{8 zA?+{0nO%hw zCKqUsOa4FJ-UA%#{{I_Br#g-7nLV;cMwyuz*?aFjB7_JTMP!z&%&dg$WF@jHGD}8e zB#C5{mAKzuXI=m6{@=gnIiCAj$Kg0GSNNWv?`OQ{>-GAaEO5ysK7wlh`phd0(+(&9 zj0T+SHLSMaIoQ%1o-b$5pvSalkU%72NN;i0Mksvgw3LczqU4zpdOnEWMS9GMC7^G~3yICjxy z_C@5|DqBUV>>I|_2w6JTR?ULb17TLgWcGW}98H#-7_f(r0MUiWY%PXs!va&0RPDB( zAc*256@cN|rQ0tS9!-@02q^=s6W0ag3@EhDUlWDbQMRWajP=D>iyzh9obbK`u;=Ue zX>u+!0Q@`JHX%Ft=!}eMs`$dE=8=R0#w!CFVg@BN6*t1p!E#Lu=Fh?cAj4NrrzwGw z;n7YThvz#LL|kFcON0$CQ||)k@ivlh_sVCfQA7-GFD3gTut-t-PVy9B6X0(P#ODWL z06@iti8CUjF(8VE^>Qh(eR%>Knb*gOF=6@e=yinto6v*KOCzaEKcPDLwa5qbbk9A z@;aa(kdPu~B$`=S3?FP9KNdPIo&;lUGxW~DofT* znDeZdjPC^J8Qx_DhUor%>_x2$&;;T?H{=<^r=Lca_ZC@Io*O|legPI1c_yL-5DkYy z@#abW9L!Xn*FJ`TVWJ5TYdj}>{Ni^`l*h2MoapbTh_P0_(*NMkvm{*7!f*{0fz3|G zdMSMaW2~10Pvt@VUd{qNwm+k2(apT{8<4Ymgwe%^M|;7gU*ZQo=D2nvr%^iK%QZeY z1KmDjgRV#Q@Cv|qHUWcjgR=cB$sn-cjd-m@t$+)JP&Mu(xf~N#AZlM0<#c((H`OxVzz%!iy7-zwx z{ExnCFps0e-h2#4r@PxnEq?@l3BGiJ$UwI+B>(XeLfW4%dBI7dDjp87jG-hJMKUey z)lB5y@5)?SWIgi5Z_a)dM*uqJh%fH)*C8AAMo-*48KCju2{L=#4rKY6@Yrd@l|9p! zyO<1fpgDZ&n+0YYG%7PJ!UEc<&md%O!02|?AASkgN9HG#(BsFEJ9M9qi$o49d?~{Z zPIVE_>XyX#ZPYp`3=O5RDAWHf$b4VYydl z1xel?^j0$yJTfO2 zGvBOHW6X&Zn3HeOwpgfPI1|OQZ2(EbrbGW&oytVAOFD~iVj)D|U?jY@W%9_F^>PY+ zI2O(henOisW}a3hm&IjX+-+-n90{G=5BDQ)F((~4)y|Vn8Vpf+@k?!{(;i1Jvye=^ z8ZJkMx4$!=__oeg7CK0+%yj-7eKUld(H9SKW9#d>p@17m14 z`?$v7gb#v`J;78Lp*SB7k{BlmPoKh1!h{WNF%iXxLxry}7?2%fm|O!)VaLKrci40+ zBn?obT1t>wMYoe_gvt5C*#VMpxhcj5)m|YOg)J-!=TTeZn1JY`aIoOVOI&IkFr&Ns z@Wd&2=jc<&=-!$X$d1EDI4wrhI$`Y3n|^*12z&Hrbp;zFw#5hezoGA;hV zpaFAGX8TW@96OpkRm9Sw2!$=;DBv5_lP4sP!TCdlXL>fpctRI~`Mv@jvkvmJ{Hh0c zoiKxZlYHV0#3eV6(u6hF7j{DMez6atB%3gU@Ec%6P*c!jhe3WhV_U@qAFpJqc=`?I zbK8ia5(jlJ;*V$eBHD7{DMFO_s+h2~i}scbEF^5f&ik-}<+q5l6Ioy! zG(@MGAaYKE7>-!x(>A^F|5s7dktwhXBbz~?JaCRnLX$STkPIb zQF)pll6qd)9_@U-VrEGsJ%ueUKS_Mz-dPaH7^M#~>DN4(k<6eE-nek|icv@fw0X73 zT_Y&ba}wG8QMz>lCyowx$R;HtX4YuH_PPgP*~oaqw33T`g2-isa}ZivO#q<(c9nur z8A%ZxZgC7T)FQ@}+Y*L~ztLaA}&X7J`|BbM0RO{hEkQ>wtv-{i2Ouz@ncbz$1feB+eHJ!yV8Bbm& z?CB#A*)VM0;?1@{!O#B84cv}oG&KxYsIU6U@KF}nycA6YdyN7dM2-o|NAL5e;FAl4 zh`uPn?m~Rpp2<8pq;I$gd5(%uo7Lj~fZGPUg<^x@eF0*;PtAu}6=Kq-;>5Fmwpp0m zD|ocJa`bVzHfpeR)(tukb}LTW# ze$tRP$BdAf8GdAU>c81LBx)jAYrlX6Q*$gVnga(W3%1nB;HErzI7lOiAr^TL{nq*vM7I z17~mzHpu7+K8lm^n7^)bMJd8x+rolq^YKMh=_6U-@XmvJMvO5$f8{s^HEAR4vPdR; z+`CN_M%{@{e_aFwD?j3YcU~Y7W%vfH7NlFs=i+R*toBL#g5w-$5!VPeIo_=OSrD)o zefV}Ee5zB*Fo$lI_iJs*$M5YwHXONsoB9IN)b%c(MZ2K=R1L73=!F-;Dd4 zKB;@hnu>p=nSQK&Zo0BfcjwvjK6k8b(;q!g(sX5WpGN}mFRe#?!*0`b1-{#DZmD@Y zAlABK)w-)Y_)&VZBl$UPu;cb20E|Q4f29ApxvsV;rXFIs$@Xz)BY5-E<<#{z`-g|X z;=IEx?=F1KGec_c0N8FlRQ#~}*^{scCNc~wRX4LP-)tn}R=r|Lv8a2MHw#ovcz3@p z&c`*G_sBgv;%-*t{%GRBeSAum&Ki9>M^ z%JcEBLDgj|yN@eZfW=w)aa{c2#K8OS8qFD7zb7;w3n~?TRqb|fU0*l{w9c42mDgoL z-?bP_>HapXj9lTd!u++v=(d}fd5EmPVMubssKX_LgXL@Ru1Zwjpn64vG*&h6&8ii8 zBLdM-McT$3^@<0rxR8@}AVMjgvlZMWUGz!xn}UEB`#3ToP4*aBYU zd*_PY7gaQ&hL+dg7{S6k@~eF(_}1C-z{h%H+=#b!e820ESmSc9t={1MxcJwhmX*+J zdzo(u=foc2-KHoupH^8Lu+X_P|23LkI;Ho1DjOF!9v<@*r>Cme8TuJ|t$alXVV-;^ z4FV%gLJmZX#fn<`-u>RJt>Z1K;>or8_-3I`FSL#FMTPitCAGB5?{7Y?{CxJyytTED z<;}2lmsGa#wco)9J|T@hoTS{LPe%t91ry&>SuF)qJ`X-GCI|MR~xt)5ps zH~)R*%02!40b|(&WUF|-%6M~dc=^Hl!lCTu8)CHCS-~42kwEI^G=pd(8Y}$r^WUj} zOK~jZKMO9POpHKjk;9(kVpGbj(zK^<^&SSTL^SNHlICuH;_(L@Lw$sYT^ z;7DQ5Pscqc?ZF`ApsKsh^0%jh{;v`$_J~BCssuXZC6r?U)pNvW{cjNFKfTcZ(|WDU z4$1!x`;$FIu3D^&SH+tD4!2O*PW6A}eA=feE=i(1xYBJA>bC~Yu@gjv&~ZDhkzB$5 zT#T2hO0BOc{!_q6^|vmPdWo?aQFLcy*hd=DLTkX*CtY`NCx|#lN127p>sjpNK)dUn zMg<(wS_Pakvo%KC#~BhE#UH+8nD(Z5H&b%{9QTrye) zPH$%puv@?VbV}iN5;3+;T)C>z=>3vUzXD4OLaz%sVee@kbdUnK>z3>z4+SJy41&$b ziw*{W00P#bS?~H6`7=ST?-Z>T4Q_ZFUO+r)I33kV*v69k7d^-$&ZQhjidkWDF(_cf z=@~m;!Fy*QpJ^Jkteq*9ZGz5}F{-e*NY1HM2t6zvXlWw-TZ3oU{6?mDTn?dYS^)4r zeZX+LfPJsoH?|H@(w5m}2ZW62-rcf0mRF}zK%nWweMvu_C$8iNS`;4!kUlQJq_lzD z$n*qSfyTz(G7+ej&Sy;spUp87D zE{c_YUuG=0ZBZ`v=0K@sMTmCuy_uwPyzSPRz^mjEdXrU z|MgwjrXl~5*`wbA4#9vByxR*X8GOnob4>^vR`(AZR_`)`4a35zC$V|p_;t;vRxd(S zzr1)$A~StqS&eY>C-;?oywbN@>LHb#AE8F% zt5xgt#v#+raQt#|PH)=qWkKauXrC`B0(`oKA~A$kP)H#y8TuIlxdzQ7b+=In(YYLs zrwetHX*K#MtIL&`skeK@DE`W}jjuK^BUv9q56^o{BQ3=Zv!|X@!~L7Ltgr8O1;WBH ztS|2Qyf?XUTk~T!RPM#0KkqXgdKlsiFlh;6#&LnCXFVoFRSF63xhHC`@oWJ2doCEr zKnr^rnu9)v4a9(l({#TKSTdqYJs$H$)Hf(+BAh>|%OC`FKM^z)Q@+)^zt^g1L^y$q zqHJ>=(3L3Ep3@%ANXxoGB&j;eAnEHean`BO$cRr6{xrxRm^njF^^58}xAFewGcsDK(J4 zf8WPOZE$O8LgcpX;vx8BY>OS1Gyr`a}&`FJ1k5t8@7nM=Pxbl`1x*ej3u60*>kmV5fTDdHDSX z;FiCM>SjA@uD(>M(c`ZmPR3Mv|JQ5(^hLl+%;d~lON#$ieA4p>4Q`wiXDN8As&Q1OO_dG3 zUIuz_qk#Qz0^){l6tG7+1Qr3(R@;H7AujJNSEg+Pzq58e4yY4Rl-zoqO|TCh!$BAZ zCB1aQAmBkev@l5dzx)LimB}TfqC~p|bl~19SbGz|Po&sFudXhNn8_0mt#Z>>egV{; zXFdc@pYRH(3ern0D)X=;0)w74K^Nf|k80 zP%-Y#8e0Vxodwov888esT0xmlxPPuIcHvoV{Dyp%H5~3D!x9~q*vU~q3A#hg*a*Ra zF`f_Fs?!HUsvVW>KjBLCz7_zbR3h{B&%nsKz7K-Rys%(@Vx#w@Lg9W!Q$j|`qK9X_hIY$^xef}BnVR%XzWY~`te1#3>xvm7BP0f zTK-Wo4}G~i-X`cx@0~T!QOQQ#=CCRx2fj$O<+XI%`ZQDH}Nd!DWSb>r(=Jv zto8J!U`iTUC+shIFuXiyD>4oE(0$)7M?jG(UJZGgz3@FqI$zK#wI{frbB&s{TU`>l; zXHpv!Ft688szvnvXfFdf$v^$|fXB0xyz{`XD2FK?+kLdY2{Mqu zoAB4cZxi+3I_dTKG~cZzwR&Rx(Bns`CXY3uzu=67py;*odlhUMX(?nyd~Si=XU97v zq?S1>Yj!#5Dq$Knx++l@^L^cv6PzuwPY~!(5li#FzC~8&c08p6Y$WUa8eyd*cyZ1S z1bNBLnr#<1UvBlxOh(Y6p~`PmT<7li}6MQP=G?e5LX0<6nlCip>T16dut z|EHIv?m)7v(UK;ZX!3K=PLnlSNoUgw_Cm{t0;N*Y2lR z46m>E^D#G0^KoV0y3ZX7T?WT0vO4qnOVtJ5<`VqdiIRGcpOV!{ORY##WOC~lzno!4 zn^mat>1MhL^)9g6vx( zk3ZFKM%tugJ$HQ9Mhrz%W?Y%ma`k}6s-+=#3lu9jPP+h>jdxy!VB1B$7pD=hw!)r6 z`AU!Bn^zl+j1qWe{*E&N>(}A`9r>+WEo2hKZafOu>=PScE^EoE&F^~v2=fiv=8&~J z33-~u)3)Swee7m$c~XNvJ7}sgr)wd5s$b^OeuT62;1_$K#c$9OFv{H9@jKYf>_8MMX%j=lc2$iA zU%#BAZe|JwnZJ5s{$DR^WGl!*R}`zbVLz%kdzlk@5OzzYOs z2bn3t=^jKj7SMFAGe$dJ^Ynh%QFfPC=cpr z0IV6v#4h->6TdANY*zjIk`#4)k&(RHF`JzFcBz}%qsm?%G?*kk?29MKvYAjGqNE>= zV$oBm4=v^Skvz?;UNnbQGYdfvgK%Qat{4BBk=?8n{QEsAFA*V_*syM<@?Na%>(>t^gBX z)w`6j>dQWu_jix|0Mhun=>sU(Zmo#PBGc+9PT;r2`C0MZE$5nd?F5e-`1$M%&F@my zJsOoVg(qDY6O-Tqo51M2U(oGR{PlCUvO{U#rI#llYq&=1fePni%eyjrV-i-k#IL_o z*_w0y{*1A4nUWD-VFf5Da_8^dS0exQv{d$LZ5vQDO9>Zg6|61}G}C!WQ>m$?bFjGt z7V9nebx|6duqnqSmF-9fTDcT{pdX`*=m=P+iIv*Ycf{_m%~E6M5K-(EDQ>xfz1XF2 z|El?n?61x9hg^s@#CP?3T((o!e|_^Yy6>!SDEW>Pcmer@_UjW6J#+JA5O2;ktSGl1 zJe3Zz%#*`j>dtct&TeL@3Q~MuVfoEBYYV6~MqMT=SIatizNfeKX@j_-X_SxH%u60E zEg8&I^!iDF332>AhU*Asm)>Dzc2wDc?55Ua`YZp=*N{X}QQm}t>CO9LTC&v!)<%4n z3Y_gt2TM&SnElZNDqGZsFQRez1v%i4^)LAC=0q0V?wN~>1T{b-;fPcb$MvoXEH&xc zwiYPE6{?5V4uT7u%E5IK3R>@?R;R|L0y4rFSVB->`)jE{Sg#|BYV)AxJKN9wmEy$M`fMOn0x zz2h4TY)`quVb-;Htu}o@JZn*6G0T44->F~mr;1j@xp&u^N$dFPQq(&`4>p;c&&D@v zy;F9;L0&p$kMlJfwJgOAh9gxO8b!IjjifOZef6N5iEM~Veq57QVaYM_{*j5)p>M4Ml)yy6g2T`Z|TT z-~|ypYGSg`{zSC)0tfj|Edzn4hoIGS?;EwqhnW1RJyQ;F-&npA-3H59mn1=e?IH_+ zcH~^p(UHXLP8D@;jTd{>)Nkk_++kiENe_6}FZg=+cUWBX!=l*1-7}LUhV`czwn(xd zurP6y4O`0+aEC89uo{DBi$^hr{LsBN6MmxZEtViJ{}?>zW>lCK-Gh;H*QVr2>a56> z58x@SSZ%udBAK#pWViuqOJB)w=t`Ai&9c1YW? z!UIsgt>ts?@Mhuj+gY&aW;Lzl2CsEcHbObdh5d}fOOhr)r9S!pW!$}Kf)9ZnvS%eg zU9#y|RMgHz0q;YJoqbN>p@%OncjQZwI&L{dGMG}5Xcf7|ULLD#5@b&3ajHnCf8e4X zi&@;^g*8&fk))QHaV)By*A+$I4EsCNXaB=Ye@UX;0odXo*!yp0xu9!okbeP>$FfOu zp+9?)=)E`4hLE6w8f29eRo1yiRQ@dgSv#~@B_3NSjO8QV7tslr=++xVrr`(|0+bLK zjNLI0Zli`ZprWxVq}~5Mkkt4^b$Td z)l&JmZ-BC^eBj2X2jtq)cffpH1Rk$&M-TMLrDx<;4ZE^S5Bc z5s}MpEfDTIa?_sxA@UC8I2w7S+Z1{!$ljbrnwP`Im1)J^?Njl(m zlh+}A4y5`78So>8Nc>OAgPvnm;a^7|t0Q$Bk#dNaWgeT>MuDEYr$lop%Hx4TWC>_X z8C2UTL0ywlGg~qRQ4E@^eyzeM-wb#dgmYa2Np2@jx)RSVZy9O}uW*g+7X*SYpCtkm z*lg)opGf27mi4GC+o-&hXq;_z%!XS<|L4ui=R(XNdBNDiJF;BjFeV3g5o$({)KYlf zJ~kKIT?`hBNOjiwwe`*Y``s$LE_`mSX3&C`U44M^SnSV!1T@`qLOD~8)Tph=1-6_# z25~QDN`5ng?%6$=jhAe2t3=2W;{&Hu!NyNyz!FP|>v(R!!0h6|9^UlFh)DtW@*g9H zRBf^WiPf6ci~BX$)Ya818sAZ#&OPl0RLb;0a^TWO0~pY6Gf1Lr^~V~#*-kRae8^EU zjMy4Azgz_I<>Y!k?xi2+o9Wjst>|l?cYHV-izwV9g6_`#f#CyyKI7?iHb(hX$%TSD zo%P8JL#>qmS;k|3mr>4$ba1!kHP0Zos8J(*i6v3MEG0oVyueVr*tE+c4V8QuRqZ3xi~ka!7IIA3+1KsTLx$demq;mha><_bUmrD5D141;(ZUrc>(&_s?uVjeRXUOsn8A;@iOD(x?ymkdGOsgc!kTy9PC&)DPP!k*wbZdt)C5JRbx0 zOcau8a|hpGb!l8}F@FK`qYh$w_Xz%>fp_Jf;h<Sm@efRlf=$2O358(YBozPmiMHBm>Cb^wjTd6b zd{#EkIRG_Wkhz1D?~ldz69mVoNwW)~y#x3p!Lo9kZ;JMj1G)P+G?z0M_U+P%m6`=2 zBKbpg}j`6s_ghun(@R12VI$=y-ss zYkSU<$eMtKfv+1q3_CO0Yc+?0p~~AZ^VQF}QlEt3jCmdTz#mP#U?(}^_Q6gDbLNd` zYb^G@zBM*%^9 z1P(+`m+vEVV4U{|ll0B~vup>f)I=nzDVR4de$6}X3w`yw3eqr9gAmZw`L&iSjL5S* z`U1t|FHmO9gR;sXaw3>W5^f z)pQW*T=4D&sdoVV`V|?td>|R&%z`sbh`y2OMTW!zAO95#3^uM|3-ZP@8!#bpg-x* z;3XT4HWq_Op=K56Hh%6g;}*2mef6)apiyHEb@FP+OvM62zYMZ0!xaj3Z__Djn~^$Y z`3c5=Hpzgsyg2BAeS9fB4b7CmMC(}&2rx>#sXQa9-4kie1`d0}aG)f-&w(V`{o~Zn zB{D6hAzXZP{r4V-A9C$3UVmrA!n0J-yvoWVT+|$fZ@h@0!M9eY)Y()L?$wsjUx$K; zgj8>BnSzeaBfTP*m9c6bJmkI%AC8?KrqDtp!r5M(x`LRfB)bdw(SH(VjooAZLYT^wiVxluWQVTplp(xdiN5{t!1L48tX$s_S@%yDdvF z4bH*{U&l(WTk7;ZUSq8$I^fSq)Ls54LwSAvr*4~`N}DIfTMvAr$U-e6CI3j@_=IjChxDh*XRp zKR@UBfr~m8QPkS|nse>zBjE&Kgd=wh0V90e)QYfs^45B5gZK;WKbg}iFv9=I!qy__ zr%xF4Jb=`(;MurNar)WQ{+wJ;i0ymkY%gR(4YcrjlWMza#@;grSc-R8l?uz`%T=>Z zg!(^qcy+g$k&UnL@5Ww@_`9)R^5DR(HpM*^j~kwJWS-{#Wu9mxeM>B|D+=%fOqcQ! zgyor~|2~27pC=%ZkE~F(An!-{0y&tf_kVunELR#Hvx_B;qA%F}8StTq-H-SvQUaHr z{dI<$|8j;i19vx!bA0JxIFb{w!r)^+{IUI4?|tZr%JUl_44&I| zlwBcF!{6-cuSHGFmuE)(9c@MU-}n9B9c_EbgPjk2{6PGIOOGy-<<=BIXM=r-m;+US zTcVWyH+l#;{%`O2H^qH{(*tU_vLfcC2HcWMg~?xF1EyK0WJ(7LL!U;lhzem>$sF!q zK^h-={cJ@Mcew8T9drLbGyHdrBqO^kEE?}tv}~v~{oa~ff@G;sI&fnUO7^UgUm&H^ z0A(FT{6;dw7=R`RBcfMEDPSR9B%Q<+goXfZE z#tBMX$Vq;aEtU2fO0XmJ=daxpXqdrq``CK^B)#|!2i>d+%6> z{4_ke*y?j%hA*g3=z@(e0XDoalfSD?dkb89RchH>Pbg?mk`!Eg9b~hjKM&+;Ll*l0 zRET>Ur1i@;W&-HMJnxCLAq?rhMNrDof3*+@L{U+&k`j}_aC3+*ur3Qt5~FWsw*9RL zNdeI8pJYn%cjr-2!9y-QXlvyTtM{juci_6R%yj;*BM*rY1=P1Cx@_Qy>3~Ch)hG2S zw8lhj+OYPZwYso!3EJX_ob_8^InQ~ofuu?7J(~k)e>hA4f?BuHXTd>2@YxTtleEG& zAu&<{IU)8O`Ti7MYPcCJ7m{+8c|P;{IHawHyMqKlpaG5h#A7lQ5sTio`Z}K<9yI$wecR+k&*=37(6li_Ty1Ax31JYpQ^zczdrx}bl&kO5C_Rm;C6t1C z1+*6y&4CM-wvF^``X!h7kd~1Au$?);Q4Q36Z9DzEb}ZI^8MEV%fDMBLwGi1biBIYG== zN5mjmcz1^W%wNMT*0#{LDd9x{b;~48l<$FrP^S39$x*~13*anIH9xhWNad1cp}N@j zdKidj3!EwA?^C9*-d3QisUjh*4@2&Tx&{qgSE!_PHuLO5^@!cc(G{xLKR`x?^s>)@ z0Pc!x8IeCHiAcE*=kbJ&z)nd?-!IG~SEfNn1uFyu@j4Hlt!%XkpzVsqjD&CEhGAX0 zsggbO#O|mP$~+^6UA@-+o%P#>Z-47pN}G}bjvbR3((fIGFpsP|_BxlpP}R+WvMd(l zj+NiyqlEm6mf)e(cUmSqPe}KcIk}8ShC5qFs@8AL5-buvUaNirS`(Mn2{!;1PO@GL zWY=h41wQi^mq*DI%=@`i2KJRNbjKV0t?&6Z&<9|$y&WyNhq>wfX(`|H6it|arQhzd zOdYH5^`}*ROSq1Gd>K;J*tKdy9HYLvSA}1vmu>}t=Qgtos){b}vJ)~N6fmv1fIsnc z`dV*7hX5hedY)PmQSq5D&ue4|cO`UCudL<~QH_Bvknz)DhKR|BAE;JNG@uDF3EPjR zWB*gi=d%2#ptL&iq?au8?o+e+ORnpemSG37ULV^Q-6NUwyC`c8g7Fa7n@Be!+uj_g z?dtoD*2>mRzKkB`*Zr#`yCNpqlCc^N8zIzwh1|i4 z`kN6w!(uW^w2(;R2Y)7m)1pN7oxgNGEHsPg>#LMk)|+GsBAyP@dS$@>?m15_Qsk3p zwrW02rjFLgAiiALbt^zOnWWJt)O^bp&;@+IAwC&COzHg;Vi#yV4&DA_6GD!si*|?R zTd%&2*FO9pd+zG$2cO9^2R<2Iw~}P)vbI5NvG7pfyE8A;MYZU-=?+Vc7ZI0sn)kTGkjk! zeN!UVbUOCj5Ot?XU5$=Y`|Cqedv89cn{2O1nyFO1%Vf}`>pCC2Xf3WU4Hp$K&C1vu zwFOB2)fRyKj^p`6AtroaY!wbP+6MirWTuNVmgE5^P&wlZiTrC7cmRz#XUqabXmVi-;uBS>%5=g{GE?~)u;dOr~WjM7r=FY z+W%LRXytHl#}$7D4#JP9s4qyv0%{=0fxvz4tb+)^{wb0yviw((q{`tH8C3jo zkY>n!4JLt$K^s6&I%GePvs4$KbN1_@aMa)M2RaL9?#neD#tfbIj9aA;6<7vbxRYgC zK|8aJOgLy3=z!mSbD7JG291N1zH=XI76}8$r3gyk5ARaA%C!Mr(gR41%d=r97@<6; zO--%|B=7dWq58}r8<(Uu8B4*f0B{nu)7-;hVeU8_lW-~P`wnb8@DvfCw^P92yESg| z9Q*(z6q$N%*TBD)$c5u@7b&rlc&WEJOrqtN{Y9CP>vCrBf_E3vd;!&QyfvpJPXeWv zQPj0kF7cZwH8uSb(8oH2@pZLfW*`)lBS-k^P=gmrY+eMD$}b#>?rb!du+@h{Nwr;V$%UfJ#`1 z+xE)1VlYzp=Zn57Vq-o2B9frPc~t7hONkRl?>0d%$&?7u>I)U*-CCz#_Mh5h6{4!^ zzX6?3oDOW9CiKxz{o;Aja-O;;X zlVC0)&XR9q0#3q#mohiZwG0_@qutM?k)9qhT?+ICgkAY5yxE52=&z64$338e-=bwl zR^+%43okKH9# zSRNUCeg^v*L^*@mSEn;x@!g?DHvns38K0W<*9FYVjxZ|-!LJ>_J8SpJ-sa+Ffafm_ zDw(KvoC?yCr5{!9AAH~glmiv*d)dcB;%Zzos@f=^foNgT4Ed2K@i!41Ip;T za?Eh2lLrOKhvN}pM@vbfNSK1`3fS&3z?ZrBET;ZuIQ$CDH9SA)Dtal|KtSjVGM^1< zH(oL)U4$`Y6vng&1DG17BJ+`ML@;bu9MCe9t!|T3Rw9GR){H(ud=|4*Zl?^m5Rnrp z91c2cM!t?99F8>(*ahUncwI42DA)GrU!OV>=1TxdP^Ve0sfu;B$6|o?!ITjxG$NF4KXJ$jp1H6Ye zA6MMeTg4bXt@#%J+Xl`JnJvF>mvy>5Ki+W3REpeesUW+6V)KDneHjM~=c& zRrZUKz;6@}EmRgoru_W%Tge`gRCt!Kc$#1~-XD7%UWkVXi3>n!sBqrdzSE`s(W)kC zam=1*s>{L-kF>6t^s(?WD`` zirde^QtyDZi+WBxgahxug=1^FGeePAiq~ItiL&T+XL zIg?BC)u%W^apg0mc{^b_(0JI-_dB!Zl7}vQR7~PvJflT=O1;p>%{i;Hah4lCK?}8C z6EE?uBX$w07CK5fFeWoRcKf=5E>7byS}K0!f?RhG8!UoEsvMFM>=kwc?q&EfT?am1 z#JTeGx31lgqaydh*N$wbg;36-)cSerHTHr2MN(ofa(U*QGYhIi1NqxKT=7qnCqLU0RBr5-?04JZ|_CY%;#QB!;$UvdVGN@xQytR(>U6I(nJ)@6a~t8^|f zGLw+uXstggT_Gg6&1~qW{__LmYo|nCYjIXCmK?Dg-ZL!xeTVFhrcv%oJ51Cy7>EMJvh&uKk>_V z69rdS5*-D@>=YZrZC3||lENVcks%FN-w$$UFW5WosM~^I`;f^oild>lV=T|f>44t7 zQ{3TA?(YR&DNNlW)u@{>m=l$h-jj**P|C*m3pqK2UupSCx8hsJDXQ61{E{qMH>9F9 zA*`|ZT0#q(2SgZ=*yx7L9${(jeOE{9vI;-30d74O9lvem&#%Scd|$bj1iVkV`afrX z(q*gh8K1@PobWdn3ng=Z#U1O~1%F-eja-2O+?WkB#_fb3JBQZPQ#dywQmrNQTBNi; zXwi>^CXZ+jWFyz)nGEj%+mgf|;{mG@`@`PJk}DYD>7z%(`~*tjIgpTJ;b0?N{J{h~ z`zHXpaBZgkWN>JbZy337k7_`knG*+~+z%m*`9yJY0C`{LGK}Gj+pKbF^-$!<|ax04z|UdLCdFmYimJerHKsK?+zE!_30SR!@hRbuzs^ z&(d<{-1uQ*=jC1@vg7f7A1+uDwPtfkwr1UbxH{OQiZ~4%t7sPY;t1 zWhLly589m*1+_cx?w<8?xe3$p&w<5pYpd=B{VujmlGDF?-t$TTL2F-RpNs zLK}Bux_f3&zk6E>dU_mAMr^??PzBJXHr!&r4sBYPS%mtBLZ^L;7iBM(X4IQF=aAl(F}U`0p)^?h83iv- z0nMVxLg^2QlwQH>oJ=VR1F!0g4SO=QEIHLG#e39qOQ!p`y47+u8cP#~mXi6!jIN0- zC_mIT5V)E&S2X-cJ!}R=up8(7C`QT7!%NnL%8k7RC{81v0w0jzd(C)g#YOWGCQ{tN#t)FfU#FUbQRFxMls`A zO{MN5q4Hy&QFvf{W>CIpGqCtL(^Dk~J~9lkDIt_~>{#NEwAKQKK7mM=0-B3aK&J$1QdsB!B=P$ao4253 z(?Sy$VmlwBcN)C2>B3d1(6vj6&8NmsWVH2XOFaTa$UUj6o{~U0xnLN2uzz_3B?pGeXO0dPqxcN&cnG{m}&$CySk8H{)j~EQk+Qt zT16g(Pr_*zqz4N^r3n-t5ey+gI3f#vU+gmDdf?hjJNLbt>>2DyaoH~&0<+#DSb;PP zpNqw{M{x-=i5ud$2Ao|+C=0a?&CwHChYWxph(S~tBtg@42`(UbO7102@B$LgVeW@n zcDEpzxX;&{4;6tCxH_){Qf6Ub*Zr~v*48P6gX3?}x&xJP*ZJ=Fje$LA8`R}~`}o{i z;DFntia9%Yo3#!elpCoHxBI7s%CPK7-j!QcVO5b>k?s*Q1w-E};BX-vl|R!L$MW^dsqF13Y?cZk8X#Q^Zt z#Vk!sN@^E8I-4T1!k`7W24_ zT+H<8P)J<^pjHj^c20No07c0aw3mlV^&cj*e1~qq)O`eaPGo^o5jlgLdrn(g~$4RKJ-G#LBSjsquAX!@Ae5ia@l8M zp~o1GJ4jmHif=~sM0)=I%4}A)U$VgXzGlq!u)5&Ib&hj)pJsN7)JrF42wKayL;X)U z%h?GKHXhYU6dxPYTs!BurA}I-#jt4wf?d9?30OA0JhK@06$n@kV<3IaHxJB=2$Kei zG1qs5`~)_$!#20d#;4(eP}L}C_o?>pXywXGC9u^~-*H^y<728uc`Uq6am;e;O&wp_ zQD_i-=~Y3UFppOy7tiS?!tK=PfS#D&WdmY27Rs08jOpODRYhK_`x`n^3~QRgtyNkd z&9!duH>WGCC`RkgJzIdjiySCM*R$b?;Vi}SI9Wde8Af4?z8AxqVpHsu;~)KC2a>jy zLvN>AmgiT|G_a6|WmCy(pMQg9pbCs7S>_4wP)=rp5<{q@U7HkJ{14E$L8;edb)&gUZ)vFqFHzaB`g?L?8{ylrcc%qF_}RTrC}j)2U)5- zG>r${$SzYvW=7iOP;<9mFFuRvFK1=E;r5jLDR%nthAHK-b9SSeQ?-X>l4IxP(WbQ8 zs7hCudTsYRBGSI64Db<{Mw==LtY;kFY*TOmQ5Bzd>SikO zP-xFNI1eJ7ytFA&@%X*u=eJ4tT_j+IUN^vn>E0JwO9>N_EAkl5*o5&F0NgiqY$>d^ zu0guOo>e{w1lD`SA)Zfm-AUW|5@85bmPT{}u0%p7$}VO?BgRx<{7q;u`Fqjbhbt8k z;0uwY;V$A&jpchy3X~p`RaE!$`FMZQ2%`?KUL!ZRFtz)|Cx!j1NHRn@!1Bmr@-;Fl@ga_0{7>*vwpLRW zpA+w>Tju0yD4lhMqpdc{IX9%9!n@^5>{Zr=)u!c7e5Vg-bDgCFM}bcnD?1$$mI(MJ zwdvU3=HC=dXneAOS8A$evB4^sv;N7e`uXFZ(8}*Y=&OgLTR7~3^z%RS32bC3DALDva<2ci~As#wL-UJ^;a)g=y~!DJKDWh*=l zNV!YaMqAp3w@gY-Z>Jen;LjD{!mS<8R?gdvP}cD^1;X{PhSUU`vmLj3C9gc|QlYfl z8}#`d&?{8E)=O36`1)S8#zF1vugO(eB95FN>cp;D+ljf79tpi^oQ8Pwhez5m7B#OO zIk~>hyJI!6z@X87@r-$atm~kP{U*!8K&~t+@d0bPmk9I0*EHi3?{l9!amcjJltx1> zH95UHeuwOS86n9yIgX^i#9ATCOQ2vNJEjWJ*O@$9LW541hcJgSNF;eDDVi2*2p^f0 zmn4{AHtIND4Wb623BR)NEWV3+&CVG8067y$MVQ*9sbtB}B`CZ?(b>9Cm*^-?Ms)mc zB_>cCW|cA{qDXlt-!Z)EweU_YdhF8?Zb$|c2G=Z3D)$Y3rPb%6MAbw=od9IU-;u$Pm zB@(mL4iRyv&_{q#0&Yr7v=@UIf&=}@W!>76 zFsECb2p<=6Gd}-lnXn;56!-FNDVa0CZP!HsTSO5UCkgn!7waKl@_lpmy{b2zNJLL~ zs^d^w__0U8t9|>#-8fNQajOaMOBFd>V!{n_QF>QLbfLcl2w}Kbubxz-F8q{RMp+zL z9bAATgk0F+)Z({n1a|Fgg(?u zJU>h-u9^*j>(>>WY(2aQ1j!Gv5!+YEzzqjzWYfZ>ww8KhfO^OViiaZjrP_H0nc#E; zstPKzML4+lgP_3g3UCM?qcmre_m$S#!A@!6a8E9zEAY2xR3va_cik$P+WgQTO zDnR?JX1=^wbc@L>|kxL-)FngFIZji=9mEDC}z(V!=vu(yH$%N~_v9K5%5y?(a z;@=oJK!}$LKjHqp`YRB)FaYM~_SnbI$W<{S%|LCxHx~dp*>zXHThTuKOGFWRg^gUG z1A*3kvpN@fxMZA#;rum&?lRES_xm4hU3pZ^-@C3Phf=3O3DF#yq=`zM22(_#G>8h7 zR2q+HJk65^QD~5&fl!H3X%b&Fi&RKTner7pI2G@|`pBdMOL|tRgDAzZwsb^P=_EqdPT9O z3x4P>ks_S|l(f>de6M1fVPZI+a2dC-LU!j4?NdZSL-;E}zL>&W^oAYrenu;wUVeuvsmZ_hmmYHM*(#NL}E?q@i zp)JZm$g;PU@}sCo3u1XDNy*W95fL+V&qol#S>$hqmf;L4!2}&5HH_wTn3$ignZpNa$WCFRJAB=7A>_L_*9X za_h?Xzi1>K6D>FGavja`yX_tB?0I-=&fl*F6w9Me&a9%3hVkhx_EZkPrtbb~-PHO6 z6cNtzFCd~kG>I$a?pgWaAQ)_Gvplv&2Zc|uQQPh18XKb6%9fcE6U?w=*wA?sajV3DE^Npgj>{-DRxKa zFYes$A`(QI07K8?0wDklD_QYgZCJwjG@noDCJ3OfTUsvaaFtMc=bA^2qOWUyN8lEfKNg;?Gs za$si9zw)BTB2&E`YF{AMN)D%-QN1-H{g)u0lTtj@ld*w&PjsbBBoEEBtS|rabiH9y z+(SQ&6ElDRc+H!eWj-8n5JR(4LW-LhQ=S8x7l^5rk=ZinQt>2l_!19s2ozRL(#<*Y zQdKNMqUjEf{jM1Bpjb(wm?-MxL2jh8rz1x97mL{X@RRq0AyBq&{a?2G=&3Aw+@;^Y zPd9!dXNYejeKx}x%0iz#1SXRhxd%-`=RXVIF{{g?sO zmR#+=hZjd5@NWm^#W@vxfZlVso-@>%83f3(j*V4aE{z~TfTC;14yM~>1f!PeAWdK$ zc_Lb(vF`ximLG6Pw?Wn zV*Oc&aQq#&z|QuE;#X>W4jBD@QTul&CpDVcU}>0DU18N}+nGxSbad1y#{+MZ>1=85 zq_bnqcSt63$w)im1;G={N9$ozAPs8|P`#R$U{|d=pF0Cvqt!7NEIzxdT$JLZFCDq_ zGQFOxf(A*h1N3$i_@Nv!`v>i)R82WbA3-QP(ztm=pb<>N35|hG>CBRX5-YkHr2CI= z=-(M(QDDk|r5zGextBLb!G>`U_E%V>b{S#(@z42B#^kbRh-@ z(uJBzKR|Dia)TF)wX;J90JMyI?+Z^Sn|kD_bS1jC++WCh8yYh<1`liB9Q}&J)vNH> z4&X;Sf%W8|?A_Z^fw-T<4r6G0@Y23D4lOd;kyXgr3KDKk#H##{FK%9pe^^tY(^LiQ z3>8RSu=Ntj2t&el{0QLYj3;R6tJZS|w61^OPIzMxUomf9KSMKNRFK395Wais7z2t(1aOxt~f!f*d+c;ZcP+7aM^MWIDx z;uGnFgY-&Qv0fCgS>Kw!0Pxw0wm6s2&ZHf< z+DeFQN-3>r_q+2aFvl%UNq9m+D*Iq09UkKu+UO)D8~8FqUmI5Fd+eVb&R$OSPi}B< zizvm9G8UjYRQa!b*BxLHC)ew%~FS!iSBHyS})dL{dCZ8y@p)sB@|PAB@T zUrC=+C0Z>RaNP6UR`R;1NLzzaL2Ns^(!1%bwGwVsy^J0=>SW{|2vtrmuYbi5nW4ui zQDj&xk%r(-IHTXo7--_AKskvL%Xd#U03K zoi>2qOh5O-0b?AyfU7u>A>#C652+;G5`#udhkrP>jU?I4-^&HhyG+NgUEs$2wcAIs z#Qz*d>frLv+cMI0sQ)|ZCksim3>kI( z?`9^vA3k631GYD9qQm_C^_uSB)Yl9A^310Vub>%LXn6Y-%#e;HQf-p7->TB-Y`P|Y zLM|Nb_qMyYJg+)_the+qH*k@7Y>**{=1e>n8jc+f^`a%(bJ}%Ma^(oW~nc4-CQOx)m$2~!RuZ0zGdSW(77uoakYYFaR^jrTNJDpfbP9;iCQ)$ zcs40kRs?3Wp)cL#f}stLR>j}<-&U{&=~9`$_~w__$KNOh`dZbI(ZOl~lcVs4*(C;n zr{!RSE>w?7ocJD%2FGk1OgN8PmK=CWTOZA_AKi0mIfGxBg&p%xjMD*_ z?12%hxY#kI=BuQG zuc^p3**HsMNOlyek-lkd?*fh`%zJ@U?t_647DDJfgo`8*Ce%V$WuSn$v}Nzd0(*%2 z+mZNx^2`3fQ!mSo2%1Wg5)Z0k5=`%MJxe8Gm%jK>@@%U)A`7t=#1u)$`BbP_fsPF8 z%hz|d8Vuvnn`UeF8+a zU1|wsA8{Q({G^Y4%|(}iC}aW9a$Xo2a)D3qVg|tg{Qh)+!CJs<;Beve=xdp*RrJv)Rt`wattuvoiOYd4cwBM zey5rBz0{AtTK=LG_*e%>>mLDs=fML!7D4HtcBg{7pcumLD@0n1;jY_=A4N|iGf?@D z{F5Ly2EK>j`x4}|n5!+Xe&ngq0>{>`u!H%aLU#~Z9p=^h)Xx<$&C)p`89Q*2uybO; z<3!GthiC1s3HU|o3G{0CxW@s!D|8ipV4PsWA082KRB=5~hn-%&c_Hpq=xo68%(@JNKnQ z7iZOXaNNX<=dWm}#$;gl5*)>$dF&~36=NRguUY|rZL(fb;FC--#x8wE1*zDg$TGQD z5`7I*%iY*CHET@t~EzB4IPAt zt&5a2h{sF#0n1_CaPh?sImZY^iD<XCml<6@;ThAzpy{lf`MmrKzGa?wV ztdhH*0h`jio^N|22d&Q+HL=SHIiwOPJdtLOB|~MmA;!vDyUjAL!Eas!La05zYjCmB zV?~se{E!G$JO;9*>7(I9Pne0#ciGo5qCZrEZ7Qb|_I%WCj zJO8%!{fLpj_dH6LDny0GOJ%Y0g3r)I2$hlH>A3tC!SI3DkS)+J6 zdfT2qAS0q>jL(>subW4T!AHw3T82z5F~_!3G}v?94r=|VncVH5H8z%K(ltbzKGrJ0 zT_(A@Nr~OzZJOA_34`d~2>{&xa-soM>3ssa1RglsR8Nw|W%J=gm|ClnKmG0RHjF9z;8_OxU}$6Ojz{l*m{*`ZWx z0-MB$@jAo7tAU&P>x6Bef9ga*rslD%SG5++PoDTx%ef=O9kQ?+A)188NBf*o1D${M zZ*T^Pw4Z3vX5Pnzuas2^E2rG&=jG7a$J?OKbsN+>EC-}(P{fj(Lj%I)*$KpD8Sqf6 z(Vc5grO(IcM7fbycXuoOegf#NNP2C&Ub`vjXqDd-bVu_|fNIQ_YRQ2(G2tyEba9ci zjXHHE{Y=ymsZ2mrZYXXVW7ac_6suM_=)8Qw-Q>r=I9OtP#p-sj;bQdetnvM{=c-Nl zjUKkr0^iv@(0L@g5I=jK^|LJ_miq2W6iTh7zietb=ivDRS?7noRC=5N*$_0{4zPty)fO%E_PxgmWsdUX6k^k_~+x;Ug zcBWY3%bw@7I}xO1$h zG@|>b_}M|k>?s#3Ok@}arsCO$E7DA8i#|*{k$EC zJz)|b&*UY_h1tI#JyUv(d`1@^NHQfux7Y)g=+m zs0E9h@A!Z^o|pVMwimcDKV4%qk1y#c&Som+fxC(U3C6^oSbLK(wL(RJYBw=Q_)bZH z%a5U^&DGb~g&8tz(VOwy9%p|Jp7X?!&1LP31&{sLrFZS`bHPbbzbR3G1?NB!xXmJnJV zVNJ&#N`0TwJWu3Ig`#OT+c!R5>S$~o$%`_ze7~>ue+MZ88^xEkdfaq#@LLd2_uJ^6v+&(V2Ep6dbuyGD4vz9!baJ0?3#{2yaAewuh{4)HI7tcF? z216w`EzM*OYba#J)_Gd1law}dd|dGI14o8I+C_T{%?s3j7cl^0rjZ_dzHv-)RY(X6 zrg=b}$YA5P(S3UUc1z)D1@~Vbm1{=tEqH%)wg!&HIwC|>zja{F%hh^~Zbc8}?XPhv z|5ML%OktN}I8n^rKu00fC1Dn18)%U%+M%l2mLwAw&NFpGd}@uo^OXf<@cmF5~T_LU-zy~uxn4>#s zdascan{klK(?WAUw|Kx{vi`c+W2?Y`yw3?D4pGq|Ubd!A!c_qUQ^jgBsiQ-c*F1cA zxA{*Ow3w_#n)S)_W(sW? zHQJCCW9vKYEE}```ABih0pQF9y7d^F0d+U)oGZl^M)Ps3qMC;8&0Nnd6Y#57v~|KB zZ83*5);R@dzHR-yrhJhvK|kQMx$yScjm>dk=Qtg*yIbq#*-78X^P$^gZ1N|E?i7xG z_!Y-Q57+SSw|zIWjaHmE|X1%yj3&UV)w18Peq?x zD%mUsga`KiT-OSX8N22k6<4)>FY2A;hJmnrWdf!Rdo#@spJ9HZ`vD0uLFVVQqV5J3 zXycjxN$S_X5Mj!J51X3^)fvCiatg#JuZ<7FYV_+d#CslYFJB(2(P@-S#;(vg?0*XG z&e@x}XiI0~O@VT1&DAJRF7{wb_;U?eLp1UfHI4=uk-D157M}APk6pdBKy%~trnvpS zH(RgXBm6<>oy;qrKD~~%9ovvZUGuBWrO<~VaPL#mcgn3v^9GHRse2=?X$)Wm9`Vdn zdE8QOkU3(mG%Ez3Yo;}<46y1p7;*gL!4;xF2B>VzJe3zpyMu@M59xn2l<|9h{_`T? zx7;O5&zOauoFrHWUb6{jgA>@uETp?To4x%+m1y(#`wowUDFkFXkcygU9%F0Fy~WucmuHg?&f5RqU7P z@T_2n$Mk`;v^9iU@pkFvo_z;NQtzgb=vu~%Ab2FR9sQp7?0mr`jt^^&M#5$g!zqTs)>~038T*W0Q92m5Lu>A-hLPmxjHxvZdJQot%I8 zKkN!1Wd3s)RzP}=T)L{ddZ`k}j^rvNb<5=W%+jSKjYb*F^XQ6V50_;5?q<1Sc%7a{ zejMd!P}u#wY1%$S*ab&&ZN5${V29C!5ZK4++{g^ojUf?k3T90<0VM?w{y+WRpGKXq z$aqy_!ZZKpS1+V{LMm&KyEDO(2zG`vj;`9zqu t{6rb{h$4PyU_HXJHvZfYA>wjo^oAGwvgxXDD`vx=ts8e3-7v5V{V%-J@CN_@ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png new file mode 100644 index 0000000000000000000000000000000000000000..3649f7a563fc0754eecc9557d388a1d406bb6dc8 GIT binary patch literal 58035 zcmd42byQW~_byB+hmuA>x;vGQLwAQDB_Sc5QWDY)l1hWL3Id`aDJc@tAPCYSNO#?} z51((0_Z`1G#{J`taUI|}oW0lDYpq$&eC9k+n(7KTnB8PQF&5h~w}_vZa7?BOvgZA1-fKE- zuHT;v9JpBRtLHzSMTTz)DKPZPjUo3Cl0;g*xLjBI#_BPw14lc9-&;dQzawbY5gUz( zUWNTJkbC<*s_akG59v=YYpz>JaHu46v5*LExr%V-W}@#(A?-YUuV{>fq}7|L#)+rI z`xdT(jT-8mPh`{U`jM#Q`>yFLHDhe71uvu&T_FlrCZzIo8V&9xYiXRZfJn+Pv0yx; z2O=F3K0e7lcqLK}IF7P-Du-n2l5191YW&X)lH0#gbnVsmL~94pn5BGwbro2=7?|E@ z{4Ehz#XzsRgO+z#xV`aL-zy5HPBy$_j1hV|+VRknkRaxo2+|@eCFzCbqf%tS+*|Iq zJ!rg^Tv5K#2m}c!X57Pn|9gZVP0As19#g6W19Q?9?~in2VlFjlMb#8T4n^Tkn1EHJ zq3W%`H2SpD)UyWeZ&5aE2WB?AQ<%?HKeDiM;agyKtc1M?g+=Podi4xO;pj}pa`sxt zVNXRe@(!Q$)5gzZg_tHC6&O9!sw1Q7m_Cg5H`x9`Q+cPxk_mmQ%`CD#7K|+bTXfktTas zmc*93p(vkeme0JyOKI5DshCYu`Z3~Oc874I&IjE>L84TAtcIQ}C4lE~^x&vX)JFsF zHxh3Tn*Za<$Kmn}y}qG4iE;JILPgMscNr zq1Nx1(%oaEI7&mJl9AuR-KH?8=b1G&D7w7Pp|)t@SM3R9C;gVF(C{;nkvEYPj!k>!y)+a!;g=`WV}7wNR3NTb)pXBs~PGuv=AWXu)37z&s0tt6`t z2kG@3^-Lo4-qc_5O70nqMC|vI|2>_O{1dUc|M=cf4PzaiXi#>W`s39qdIS1A%XbEZ zf5*G zF0yN^TQws1kZdyR30(8D+KcX@_^Xt*Wf!Bz29Z^T@4KQnwPKwF!|^cbq$m_`6Ph0w zx@5<+>GBZvp%&#)aG+MSid5k{Ad9v-RZ-u6LH`>~2!|_3LBar>^STY?^x{MTA*J*ijp0spGs;}N@BQE7No2ENad&8 zQ$`b}!$b)dR(3LWVb)u$MJyPp(Wz{yUTl;)4Q~&0*ZB>z9i)yJ6khr)tuGZVy;~yL zlXJs8<(CVwklQWp(5hzLXR&8FOwmdCk|NmK+gsdA)jQOSvRvP5pK{EWs~3=Wp(Ur& zsUKGKvYbxKu%Oyx`ExI4s-Q00*vkzCwR{rIbWQs*=Cb25YF)eXJLL{~IU5vXY-6{_ zzL#^4^*kkcs`@eNsoc}lr|6?x<%ea-W1M4#8x)6A8+sf48-k{~?d6kACg6Lg4*_PCKUhnR41J)%nHD9 z9B?eKtw|kt*LX;I?(<;_5RtuqAZ9md^Tu+?CdILKcr8P#lDB`d+u(8I@aM4;VUx^( zH><-FKc?3&nQk-DG7%*AGi5X7s=ZTPREtqN&Y0yd%;4?g>(hVz@zr7Gz03suTsu>n zfaw3r$6-PYRY%zRQ2N(QnXqsGzG5srWLkYj5_4b^O8jL&rz~>`aQx zLxFS$Eu)9i9Mf#mF;i=LgdN8-jI!apu2}>pXbbmw@vIy#t+_3%ST;}Ue%rpB6`9mp2Z^Q!h^0(ZUM^{ zt6!f@{dOBkT@e}Z47Shw^HA`a;0L27qslM!ey{yLp5ug*&_{pBa2*f%J-^M{y%^c8 zpdmPjGI*I2=riTF?7em1D=_|)BXD#4LqelklkjY2)0gLK5_S@q{teI7{DqqzG~-_O zU3dpfU*zyPW=!>&^rc_LUs+tsTx}cXyfu8gh-?6Nfg8eY@~j{G|88hKmnxQeCFLn~ z+4jDT|3yLsp7dAgw1}C<_p0kRA9)3q`mMeFam15aHCg3pj@$0o-Z{^?_`@}SQ9t&} zJ#&28Sld{%q|F}Uq~n6O1;w(I@l5gxau4MsqsR#P`BzjqIK8wt2PU7nUb(6){_Qa8 zR#})P7$wjqd`dXWe49a^!HYRg{dKAbt)Zl0;!KLPvXbId8Xu2@%SHLFc-i~1qOu8Q z1GNl+`5D>EsCvQ#b?wZKR~|HRiD?;iKpM<1b=t`kZAu=N9J?FNjLqB?lblsH<(|tl zYYTimpRJX&%P}JAE@$nW=B)ojKR1ZcfZ02h@?{h6`EzxvfupRpnT{E8A7vjqpMGgI z^Z2Ug)k8~m$zNo>3beJ3i3V9KiVtGiJEr%NE-+5jZavyjXqK_yJeAwa`%zL=zSBM$ zT83V1r%I^)C&Nkf`Ora$oRnNkf2)m)O>O`ESHjLKA%=a1nKtw_BQ~G@Ewz@=mVvl&8 z|w)nL7>9>u)59aM(kH4-osXO}K@YZvp{G#GW_nt0yc|c3f zr>ni&EjWkwa0JtwFKU&l)>l@3_v1WBdyr}rX@vPnZ0dE@VV`sC+|y6%&g^qXo?l%2 zj^i^Zw?6*;_)!$ci|d`t9|q5+FWS#ceo|I_DVuE$*r&=CU-oM={WiOLK73>{M_t#l zg|rC2l!^%%i?EI8B~J`r$GMC%jQgebP0ge~zu%G5lr!RvD8+0*$CBch)>ffPf>wMg z#XSekr<6imBD2rv_B#selpLR_Gi5jar1~}M!?rm(H6m4_kz<}?C;sR8QIGJTd;cog z>OJnyCbnxXTS5n4{%l8x{1Umx)nMk(u;f}+WbZKX;d^tshB3xxw==f$jWB!siH7&x zW-UJShw{g2v;8)o+?<;lOuWzcv+JK>Jp2B)Y_6iw*`;L#H&gP+7#BI*TPA2eWc0*KszQ4*W0nbM6-0%{^DX9zCf!D?0oh5_oz! zHk3O$x37HhbmwqE(AT-H#nWe60PlgzwaM(4!R_x7YtJWllXiFeN;)(&BsDIj&#G4{ z`X|zy&FB8!S;4szEo2eD`4B0DxW5PK_n{Y3={KYwzfWWNOarcxiDd(z#-4ue(vA^HBfw z9+d)ILz2>wQBnf0I+kuvo;bPNI(w{3kSl->FkBQ3-I0(;7@=QeC2fWsF#ounu7QVv znyQGUvm=+em9xbYE^kK{XdWanZxQg;@rj2yjklwNle>twI33~+5%3=Rn469Uaf^q& zIGur-CXI}<+Y=fAE*>r(Itfe~8X7S-D{B#LS^0m4gTKV-Y&|?&M7X)1J$uIWjGxQd z&4!y-SXh{whmV_&j}zR%>F(p?VeZZ8hKfO-ifGz-KXEXWwQ~e#2A&}y zC@dg`xc~omoi|FPV>Lhr2-AJth* zA|b(%lw_rJy^%MwF)s8*&aQKBt3_+~5-X!dIVJV{2#uy=TxV`b2~-So<7dy$v}Z}T zie$O3bf4wddu!!z>G`y^;x`4sD|XitHPv3{S=(es!ZwvPM>|u;hmPA#N8~w&rsopp zmTD!5jCZ~wqtGCsNYGXJ9sflc7iwP_$X3C*ClDL8=ufqeB*=@VoWg?dZr14SdMB@7feyERv&7T{B-uu z#-SE;Ln}nSQ8~<*-#^|_#D&hnlrTv1;5Ex0DsmiJ7D_^l>ufic#43Lr(=7>m#z_*+ zM^@a{!97_1H3iV!RQMT!&QuSaW+n?~?mP*hlGAXdtEnPz!4zJ7fR=ks@{mrW$OTjE zPE249hiXwb85*=WxFj-6?lM;E1ulkAXHG--n#!zZlx&vH8l!Ix3x^NJD{-Isu)C!| zB(E5y9q^El@nFKpl+F0kyi43_qva*C)DuP~qPOxeW61BmcW!jyXFYIRuC%05e=k5Q z4huqpWiaKn3re%RcG64PejP;Mvq2$;oDnq{lkkV!1Bq_KB^%hjr*KHB}&WXhWMkv~=(jq}d^w7Z*ejB!wTk zYjLMQW4a>Ay-S42Vby_%4{=L7Vi|Jnapc7d*Ci}o#4go7X6b* zh}anI&BpA$|48Y5!80cwzTT0$?_&!uq~k^N5NgjraY2Q|g3N%p@Ai(|PRKW98sA^; zYeYUXsB4YSOwWUn9e5Pr-Jy=|AEmy5due{-AP#pZvo1=N%M>QdctV3lzyC9@0J;d~ zVnoU({SnS@el-5hXv=)XW0{#CgpI@Gkr;Zg2b!t>u3W?6xY3AQyF0YkNTM4_X;*3< zrK&L#7^0O^NyENw3ey*fqKK@_e`Y890O>$$%gvUr>poR-OjntQTqb*~x3lS$6Y(ef zIK`fz$%_fPV^jKFe+dg(#3>!vP%%;T_4(ZOo}OR|*d^X$6uolYZxx5`D}DLJYMCn^ zRf%px_XaRS_Yz05D|@6HV@3sXqUT?=QwX}IqdL(?Cu}!rq^H%mcRic^qVy+raKhgu z;8J_?BC=}Au;pZLB=sKkoKG&-yJ~K&)3RYV5$pbl8Zpn@@@5K!BE8-0CMPJCVF-SZ}`;1o-Ft}fq2&>z~O>78!6Tx@t;&0=7RrPI? z;0?9Qbp8eIvL?^-mQ}1YW{YnZRO|DR-25&xF%IXrDh}Kw=`X%%L2Dm>sNbWNQ|SNj zQ0=9J-t$+V`q`z;iuYppiP_^{Mc*Cns#fjjl}#v!_YwN=ppz2P zW3+^u8L0khiH#=bRk;ol9FLaD91(rXVZ?v=n?ka0S#_}8xZH+i=#?MO0&c3rh?|~7 zmT5Fns@4nl0yn)gkL5k|v2O}Ri#3^NZ0=d>qn_ml(eBj;Wt(2(f?B?U%WU@+N4{_h zYMrU}#%bi18#~c-subzi)JaBeM@*k}eZrj(CAJw$URx@hb%x2w-LiqSmq(>FQd3tv zI?9)T=+0GMSD<&+@%pfv#K6gbpP*g#?&)jX0q$3@->Vl|y3``g3EYw<^}D;-;2tJ6 zjUtWIw2?9@F5=wwli%mMhs#22oMzo68V{Y=OxK5teI;k!s98C`KU9MxF_ok1RZ?D= zS^I6{`?h_9n=mK!?p8%jGgIio=2T6YVU2YJTTGx7uoNUT24GUWjtO*3Ws6&=$aso! zLC+L|auwxZg9umu83dM)X|E~@G=HhqP|ZC0fM; zgO!sJ@`gP%ha;E|yG3|-i&JWPvCQPgcU z;25uXl-?llMeEi)Ho|bhF28fPs0z;cy9brEc|b?MlLJ&@tzOg0yj0dg1kL32EK*y> z9fF*Yph1ibylCJ$PAK^M!1j@XYa!R65w;LHRXJksF8`BQOQzrC^{+%-F^TCXurhBi zX=6yo!jVEP9>2y62QE7_E2uKeqERt+o6}*m<##1ZM>mGgdA=_Vo-ztp%Ui>mwSpKf zOhye|OlX(Yj9{0(}F!@qUp z5JX4G7QCJi3T$=ghb1-TVmSP2NbOB!R|vzpTa1c+rz4}mZCUwItu4OebY0xl*+k25 z*j-bW#ZK9;arj{EkX6b5&eud7VIp*dQJShuDA}H?Lc4THx$H^bCZfXM3R6TpLz7RS zQUmBD{|=>y=K63^)Az}ctd|&_v@+qEbIs2!I~I%5O_dkB;tC7nMS4SCCdqLD#d6P& zsD(uOn-bX!Q*PBuC@M6sTmqMy`KRd6D=KNNq2Xs-X3b)bvyHC;PUg`?PM4AoiBseT zqCyw;mcOsZ@(&~y)M?~OxFO551YFwAHu`j|BBS26(js{2tptwfVy7Kw+z$d6sUX{wkAJ>kPA4lEMBfkUgztm3x}zT?aGRBnfV|8dUgO*u4n$2WxQ}# zK%L*QlZBLKwUyG@ce5+$+~-GRyY6-7{jVSJT@$Jc1qWR0DlG8XkH1wKE7z+Tsff8e zKOO<cGCXY{pQjv)|Sk}KrO(Jl@vTO@=f=~SVgjg zyTs=zGo6@g-?+~sWSbjjV}_OAYO|;m+))0EzpEJ#KQ^w=%$H8HwLWDIPvGRW(O`Nl z8QAR4v~ptDa(R@Aohjm39?Kk>eR#5%FuktpOUtl{()rT2;^|nqR~DUIv_XQzS?Q%Q zOy+d6c0|2qK=Rt(hm(ZRfI9HvL!;A=yx`8G(#EL)aGo9NgLf@TCkbRay(r?RKUgZ* zZ_hUI&8Bf0E%Eaz45?34KT#nQa9>dsjphEFla({_JCgB7OJHE;hnhG`OG6S#ovlv0 zYVnz~D2IKG6!nWUc){b)V6b8M8;rG`xS8pc1DA8;$H5uzH2WAGwcBNmejm>Te%cs+ z$1b_?(A1~su#bPxz$VA1xN&=|BHd&<)2ZRx9m`!t-(O3YlOHVKIk#MnlMB1OVO(MC ztt!d!?S0K_8LoPMIALa_K>fTH-~-kQg#;??-%+MYWZT)#W|Le~fQO;=<$MQ_tbt6# zosVI35xLK1eTM7Qvqiqu%Egd;9Di@ZJ`%Xwd6zFXY@udQ!ak0a_iMRUVL0}^`?iU( z$%-t|<~C$D-(E8E#h)o0==ZwRQ7TBGC?o9&CmDD|%f{3DQejyy@O8Y)b$OLtVS7PM z3V8(ZV*%|t2UCtDW=3Z1$mNL1`sIM*m3h?hMn$Ev7J~q3T}6j@?99IYTLWJEiT5Ua zS$+iUx}}dlzzJJI*heDesJ+OW@!C82jQg}|+eM%K@A-) zJZ-DCTp=l}i1cS&yWaNF!TQLP$**BpPjHvtC?#(SS&+4Gg!ti{Dl!mNJgnqej2)ItA6>-YKY zPV^;h<3g&myR@>@yR(X0)1OMZ;_gi+b+}Y#{bBaSD${@Wh|lUmZj21V!ryx zcZ(HFrbm5W_b6te!85_r;#(*<0`AcfYHXfs@~XZgPucA#*CckxUGyXB)ECCs*Y(*qgo&vfvbbsF>}!fgOS6cCyaOll&A;B} zCGLmepXE9w@z{&`_8%s{imYb2FNxzC?kANhXhrIfDQxDmES3hA>z2{AXNwnZkPSp& zbR2VtOg~7UUukE|Sl9AMOR$GrJ`(bCG%+$?{gqBEcuruC`9-(%%k8l&S&~S+sA%I2 z{%4H)=vVm>R4=jgyd$ShKRM4yh=$BI`F;N^#?fy#`lB)$kDEvs2&wlowf~NkYI8Yu zd~$_m<7$qd6`AAD+7BJQHUq;8dy`$d(!1pq?_atysb{f_>_}ShJ5|U=5xDK%!7`yJ z!(bqYF^!Vt5>CF(Q*|smFA;M@Mu+@b5S#;mmm!_}o|Uc`IYt;^wxQU9qDIi&%Bnm9 zqi+TlndG9#gXiu8LyNcQ8y~&uq^aatx!<^ExjK;TMgCDyZd1cH;-llVepl1qhCiD; z&A(?`%>pm=!k5{I^vE|%oHhm?tn)p=Vu1;Z)UF-)X$GToi<|~ktvT#H``4qiBpE%I_+yD5C;NVG zOx2V$b(_DJPU_^?9xJKqFmDs}-g#`tKg}sHtuCaHw6*x|dB+Q4BN{5d%@1kue?2=< zR{J2pvwQy3+M3rQ1d^u^;@IB#l2H3m_|VY!-cfRhlSyTN$i&;14c!p&{@V#ky z1j|(r;2W3OQ4Sv&^7j7f2=IO2Q`jzsw<8KLb;~EbO>k!g4U`t>v18#)~%lzOnEzBp>_5cm>=iRtBI^WK|$ zk+Pcl>lFJcuLV6l@dc0#rjV9jt!4Kea-PTVuTLjR zwL;!EpFL$ICzSs|VQWYte(*y^x5?LYW4I`0WjL|P4g%LL*O$&uhYG^h%M1OX!`?xh zns{{cQ;Xv+mi)<5E&8Xw8a}F?{Hbgi2ADDGp*V=!QND*8asUH+<(5V83Odawg78o< zT%_Wc!Ct*bk4z{%ltA$q!HI+>frY`1^e{R>^ch)BH>1*t87fp50XMI||MK_c3)lUX ze%&{U_#$U(g>kZ#Ki0sM-Kl$ItayJBg%^uNrZj zYtBB}oRV4oo?c?|#p|25*wXWh-EN(H>99?2jz=-X?D7hfo+;Ri)swfI=NdfOKO`=R zop4`Gesa$3OJBTR<_@H-J4}$g%EKgP>qx=Ry>o00857`rGV>#}Aq8p$ln6v5J=;p2 z$kN8>1UF~Q`fZChET zOb}=4ETJU3+f0Md*oZ}}3?39nW8G*FdF09u8m8o8f@pxLip&B|!>9sI%-J5*vOEFl z&gakIQ{9hG_390*FHUx~75Bb(qhod6!Xo(*#hjxE;!KjX)8CnTzSxlmwm;xqDhxyu zm1Y_sT$jXfH7FhtsK4S11F=)?+1&F_n}$m{y;32!hmsViSZ^^M3+xqSL=iG8OJU}- z44wU@hRMx6z`#ZUX2t^4H&VdA;3<2)!51$_qpV$*?Fq9$HOb3^7-S@S>HeJyhQ#UQiyq1ZMH4tC{l5m)!HU)>Uj)2%aV95(w`%KPkM-H zwGs4j4~5%8tu&B>;ZZ~Q-TS}F4byT}0a%B#)^$W% ziQ14PShzc>^x@aG>*5yV`Vw`fNep}OJ{+n`=Kfsr(obWkkTj%IP!KP=*$$Hw;9fje z6p>K66_(o`kAhzpX4q28H^b3-avY zFpwGGMTe8~Pgv++$j$b>z8f|A&M5sofFJcz0?uFG1>Adm%-6W{6>B{NS7Iks(~7B) z_c+tJnf0O5zPR&9yC3V`?FEbXR(&gT7ZdX#|&Zn+%-FuU5y9BDjj}2d%aI7t{}Fj`ReRXN5#1-0)fCNZkTkcl^FJRw2lqO z1)OFdq3l4BL|hVDU=_(SE5M+d%(1lm=@7wD?Fr!+jyWV#JO+esvq+o`N}TSMPCC}U zI2h5`n)99oWbef(JV$i>;@w7KAx!s>h6r`cn1=+|Mh9RVWM)2>{xnD75bIEWM8N)~ zOuI;VFWus#1m=2@3@|~@zCCx-fp#EDupac7#}F48NhK!}7iA$u!vQcsXH}JqIAm#n zlcFKIoRzM^1)=yKVr(_ToC-3BHxR=k%~ISM4+FE5A^h^n{F`$697KAGcOwodTc1f7 z%6||55=s&sAj8X6YhGS6@a_l71ZVJ`Y?g{bHxS)8QIghJBoMn*|5=ZfwoS(^2;@sbsDPH} za&nm5@zEDA&gTb13bjwZla%XKkUX#(4h0eB%?DX4N-qE(&7I9halu)#fG36wYfF4g zzP0e~)2wngb)XpKvl&+?HVgsC3g>PS3TXt>($&)s|2Js zJ?uBI$j4E-CTv$DN3Tpf%+!1S7Atl-hoLk;a_Q0^K*DStC$k>R2vp}b2?;hjM9>8R zoPuo-^2Gix;5^{z#%>t9@3-Ra3D(uvjTGZ#yY;djjOu7ineX1M(0^Az#&3;?#sDx^ z3l5XnbjMTPfA-hdV%}%(8?z25u9WGPX?qcUuU-QLPKwMd$a_%xh5O%69pRao_wan#e|J zp;Z=hj<@vDqa4>RGLH^?1w|#Wf3(iVGpP_-tgnC=?MavVP5kMYlwX6UmGgaOB|~-ldO=BBL_^x7b}Bw zE)x!32gfK=IiJDPT;Gkd@kCmgrAt5)KcOOIR!MVd-v#p{jUa3FH)`-0Ej{k z(b6Tj%?*;{4_V+4WfLr9ul8kNat({|qPDvWox1tbx4w)Y&$uUSGA{j^Sxf)|9Wk`Y zmIHq24LX%K{EjfSn0MmQ*0e%rG;y-cUq=6(HgwByWS=(diMlq+USqr0)=x&&hxKkx zAi|5{@5=njMCXhm#XJL0zVKv*rwCViMGpXbi@=Zt{_&cL=a}Ao&kfyjlP`59@_1l?Ft&H4it&>;N*U^*?oMScqn~a$Y=L&81F`^}~fYKv4a_ z+MSI9Y(qew`%GvA(795}uU_7d&>kQI_sWLM*~~k}Qy*2YP`24tvxyOz|5ybk_y_PF zRs2&efi0c`!m9??7psz6Jgc1raxrhuw_E1Uz|J=x^Ja0ZQee71jw0?$GvIZMqIEnM zHrxF51t`TRpX%GZka%jXi zap*p1hI<=i`tiF0GmN5=<77X7@eBLL1ecz9Z%%&5c?nW5qLLPVLh+CecLGCFDj^p* z*HfR(58W(d*3(!d9C;B`zL5slDJ)tqfE>gC^_Mw=8DanZ6q`~+H(G9%J}Hc5`(uwq zmG|xf0c`L|BS`I+jTK44amerIwS}N}_oW^P3LB(H|09!fkp(iyp`r!fOAP61=C0!YXq`&Xp zE(Y1MEj=R2JhkVt!bTaQ0ih$#OP&)T{|gNy>7i>!G%P6^H^^bkrq_VnBgrxj6Om)H z-9!={{BC~WO>3d&*2>eO+(Z&)D3Y{As={d+5Wa@6u5kKAsW9+01~*a5+(%Rs7mr8w zCLQg@lORHr9dmB-KXH31D>RfL2C$fZFNJr|Vidq)0v#8OV!TluuIeRrjru(Lc)L%= z)k#VGp$u-cOlN)XJ2$R`*eD3Az`a66paO-V_S<_h`T$H5s$~f!&HAiTLZL}5N9@&r z&|++sY9`=fFK|`?yk1VhC5V86H>vu|QetQuCkde5bp4aTJ_brPPDQ`Z#CQXiPo-@w zOmlOj1ot_Bo7x`MjuZspX<*z50FgtU6DD^b6_=7i;67_YCG(f@ULpo$Ohe7XSQ zKl<#Fe~}2X1-a-hT_(tVNnBsJzZQMQ1@fPm*AMLFp(=snRCO4WY6d*3yRzj<*`&cE zpGoa?&%DEcGjPH8?Z-KOes-k?Jq;P;_if>F(L_e<)6D`{r{gTP%GM48}4m%TziDl^JI z0fG3(*h4c;Ind;RvtCizTma=uP$8WahK@rn3sUlY;EQz%<>MkFatDD`oIypBc+tOk zpgy1jc2d+MHXlkDxvEgQ2qr25fLt3Kz_QpkkgG!ZY2@{B z8GU@38q4f^_Z|4`5m?-(k;A{grXg3k>DPP$Vz2ctpVe%yj5XqvQ9BcdvO>9r$Krm8A1wHQauZu$fM+#=HPOuMT zREdEIqq4Y~=H+H=AdarF8B9aj26eA0ugytXM?#s_ao~m=L?(g)nR7T8c%>t;ke(XGwz@6WG5Fpvi<%mwz;(a}Q`mz-msg5&^C^?O%THXFc_Rq4 zI-9rhKw%O_hKwy9Q7ia!>j(csvq)Lr^`dlJQ1EcOV*x;j(gE8~D*{gIdE&7I@uQ3^ zBZJQ_=4?iFZ?n3+_xlA$KwiMP?*|Zz9jL9j1r+qge!4&3DIm?D4Z@Bch-e#HeWxJX zO_W*S4gS2#h)gImSg4SYHbUc%fhLb&Bx!G4Nq*O;i35{#lH+_B1NHL*e6`p2mAFjT zXPZPnz?bNgG$F*kTmz~i;h?<4diTSQMH(pSDdr1V_mTSk4o0_advVJHFcsfFUnJD4r_r}7WF*OO0r+Ng6su{ehP*D|G5Uuv4OIQ32skX9Q_uSyj z9r2j!v#O4-FEe^~epDG$Sw!qM;dvj44@xeXFKZRbPnm!5+Is1ZHTGkFp)=+Q#p;Dj z&b9ZD+}$aeK@IW4l-PX`U4EYKbpHMI#qzJ~qB9E9g)D$}mpHe^5xYf(cDuqtZrkHe zw64zr0EK$=gAdYEk6HqnICnk00|MRSW=($>-eBrn0Z%k3BX) z$ewYFSmlil-pmSLX8gXxB+r7{=k5e*h%W>6&=9OHQt$<6q3Xl6NkKu$hw_ZQ@V^tLevY=U z3M>!{y6Ioy%}j%n15rX5L8&EGya_GT1rP*>-*{i-=VXrsU?*W9?wCD7@dB4*vjK59 zI)^k$jpnZi*0VInaJOY`QHGoc1Cl`>6fZ}PSYB6fDc=U(0W%nMC!uxfP9nM|XcCcw zQ>1xA{P7+zpaBX_>4KfLo}!z|&w=Y(5((Nc0ja1L$1Wb$9Z0#5f^QnUI@|qS8DX2V zO_?VpXVv1pDXjE?+#=Hlis&d#ur*MdC^qf@fD+_;gg5!#%))I4z|qrOg4EeCy|0BB z1grJ8BlNVe&!7{Gtbz5eD_AMKb?~N{O7hcZ+GzpDpR!_T+5N5yQOv!$8{e5Yhl7HQ zP_~0(L|w$hj;}2?uM2AH_%P^gS+M8r6C>uIUu}>0W`MGx`P4Djrh3$ zC}=2m%VMT{F7P_L&S7fdXZ%Y>6>X3hm~<=1;`M|D0g9`BGUTRL;NO^NyhPwLaSWht z>E3!A+0loOa!~HONJnFiLG&EnjUGvJYYONso7uxj}ne zvA`WhWDe9f=qA8+3Td(`;L`W!Cui#2@-^u&Osyn3xi|2RtPZ66rZ6?o{~{Wip~(nN zJ@N+f&>s;zF*2DoN;)=(mGnl&qQ;BewQ@m}`GEekF#{-cA<(CSGvm8Nv@PW!lwmuOfh^HmaK%_Tet0m}ZK8I17oJ{hc~Ur7(6D5X1)H zCG4sZa5J3>`!X%EzG|76$L^oNMAWh1P6+MnocW1XCQjG7Ff@oVY^XMd8a1oLoO)rZ z0l?$?hK=$IlIUSb+gev|lsQ+>ceLLfQ_*_9+qB2Z5l!k}6*U_Df);5&Bjwp%NJB$fWo|-i1WxGN3ppAsX9gqE z`?TwC%4G}PCo^*^Ts*ysF&009UK;bIP!SNGWI^{KRBC$n4EE6zPxFOV_NkVpA0y>h zHFv5lc46u0Jhr3DXM|}7-EcFgXMC-j7kHX<+S&D^JW)z=ytW0}!0C3PAUexnY&Y6o z>WsR1M2yVHF!m}==f%s)?0J9cC7CGY_aQZ(hROam-f=TAM=(PF_D+(OXjw0YeBQ|x z@pDv$+i|Ey$(NKaE$Dh*e7vVK8*uKp@gqMZvosP>txCFS5S3p!l#&aPheyeK>K{cI z>bg;8qYSfhWzR!ga~!3pK5g4St+;rt6mpHXfgWIBu<(`*uxT5}JsHdqw_E#>U;SDg z`IV}H$Rdk?5F#y^guz+D-^poFgJ{{x%9K5!tXr)#;DX9j%;|P^uKlI zPe{F`cK}YkhZ?z0tB=|~1r6AN4hjc3;FeSY(VXxBd~GT=iuCtAM)K@}mlcIy$Anw; zLf64L2}7|6nd&95g3t&!Q#*UPpmsKKSQVBEBRMUbUd$pk$~<#ol(UBls?4C15-p$` z;S4Yqsj`9cVP%7^h>BGy31)&+Hr4D9W68-oV4>n5U~W#)G6I$mAi%2%;puXg`O*Ql zTpmm?XLeHxxL5De6Uc;Gu11AlZUz-Ovj?&qn0*a;M&Rs6;UhzE9nuIL>M zWXg7SiLyuf)(DkIrioEzBwh4l)cagcKLCy(;>PwPP8Sf}a?%>`C?(GoOmr>H>c^px z*q48zG0#9Hw*Z$MnF-3(`6oG^+`7g#?P|0UyzaLv_SO(C;j>n2K4C%)xNU z#+1E~RC$Vv4ns@2G29M8WPuQP2S(%N(ex;2l#kSbnH^p-FoCj+Ee7dVUjJ>w4Er|_ z2ONGHC-n<*JT$Usx(RPYR^20wFch#KMy!QN>lp(k(<2Nc30%LBeZ0^SA|oVNgQ6RDS@1+=}ok0dP=zjv^h}_@|R2f-b7ZRL3i*m+HBvjYT07m zY#8&mH>7bIC?Z+;lJ=Mtds3~ebUs3&Bc_V5F8^ABUY8s3dB{*f0p4HkF(9Wr`X5Z3 z^fHkWa-pd2eg+I?ZEA>tY77j2@1HJDnf83x4cz!mIN9J?@m7{^-k?mI(cvB8g|$4+ z0x;h$QyvJ=BiX40TA{->HK|!oHezI&_wjK!DknjgDF0;L5 zBF{@S0f$M=$6WNG77S*$%=8{XZKVf#gzco@fr5}I1efXVz*v9?ok-`Gt)%2@ZCQ7wt`ap@_)d4=h#MnGg#`HiN!(;JM!_go^=Z@r4l@0w25- z(F#gxDqBB4e%CN(xrIgmEKe2bmQ>e``gLUz--fhH84j2*RY&-REtn(L>-_xV5lAcm z$}_s>1!iFYOg|YDCpIqlP6RMCG)+9KEQE^KHX52k_^=oa)QuG=MZNm&lP!b<*+BG) zOjIm_Z-D4z3gq4U{tChtkc0!b)%}zBfQsr9cwFHTP~uL;yJH{gY4i z!4SmNw;zP(ana!6Ls}qw0w(VCGe|f@8<_Ck(H?QEe78 z;+rN~I2>vt&h|+9lnWH^tn9(#8(n~kqoBSxtY<8Xf=mg{)KgO~r~W30x;qlT0fc6c zDC7zDOfiT^ZNZj;otl4;S0Zfn$Wy8lPSXf|*I{Q^y59qw*{Bq;&x$f)aeW{45v=ZID4G03PVaa3aR9}2nrTKW(e0X(eRZk z5cJF(*nE-w#fI>^8_vWVH<&q*7wjoEiB<{@bmMmI{9^%_?5oxkR-LtI_Vzqq5^%blvlIr1(|bUvrZi>9f87Y}*Sqiw5KhIIfl0)# z%G0%LLMrXKf4j~NVTzB<&`Iy03_*@WKAq`l4i1>vp_Dex>Bd?9)t9+~9zqE5KjRb`4AMqy zcL7YY|9QN*(f?*Vm?9X;hEAAdZy5tAcwk@-L6J}9F=)FSE!SheCPh=B19UO3n;KOl z<{yaF47T+p1sU;g2#dt?2O5XKMZoE2VoC7^%|!;3;+)C$UPn{)jIgpa@nuD%W@YDCXLS>OnqZB0S??hnxai#!V5f*51CI3(WEc@^){+=8<6xz+Yt`VZMmHSzw$XJ2vF8Qd)bG=-0#V zIraRXN3fIMA-HS9D%?0(zj%bizyY+6q?wmTA&jb65X}=gI1<d#AcY6(DZebES} zyl=eiL2XnbsI|Z)Aj3Vay7wgXe~5s9Wf%G7y%>jI##z3u0QMN*Ks<@SUJFeS^8*ni z*uYw>Vc0{cDnPrc)X>|dKzYMMbuykyZB*?4lD+-_Ehz5vN#>>C+2IX7s7AsFM^{ye ziYQ}4q6ZF^$R{P!87%sgDSi(agBuf%Wf%6-sp=;Kr!=T|J!wFt>Xs7KnGrJ8vWcN} z20P+J^XO|IvS8uE;RMd&hk#=qdodKiZl9$F?b6_>0pMqTU#Vbx3exj5isAJeC$Ztc zyM%*6lNZ)%U)dxN$ey8>f_%3Cz@j1gr74}|eN_kU9wIW$yt2RG>r|b zMUxn#xf!mm#E$*0s)X4L!o-FbUBrmzeEtfmc8Ym;6$dT};1{D1WMwax8|&7pe)lY< zPL!)>d*0x79&^T5CmWG@R(|&6wW+Imnnda^^?|72l4_&Wip@BU?YbG+b$(VsiRa~R z^}!2UUd6Zz=ks;VyIbvt>YcByuh;!0yF8u;t76DOimBCf3gzKw;ZS^cV&@LCi2BPb z9TBEc9U@t}OHH-R_&3EfAdPnz6uk{iEtoMwlr7sg~xJN_p{j&mG8@< zi=x{)Ma$mWjSNcQi;V-d{*-0`4Q@KS%IN}Q=krXCbbeD;#v7Sj&&vTxS+=RJXC`PT ztnPnadhW68_DDkaLgrb-U!C3F+GeJ(0?1R8&JsZk6$(gN_&tnHWyGW3z6nhsFr)aZ z#G+f8`1xbUJE55CGQX)1T7DAzC;o>~A|7*r1`hr(m$y-K2D zYc5Z1xNBpc%OM!tlobHSO^aj;XgjE$d2=Y95?Fn({wGrmtc-74rjeioeDl(&vb_v{KHj%|H2>R203u!+VQN$ z9G1*k4zW3wR(89Kr_M^A8Q=Oy0m1-IiVwnm6SzHhTu=0EK@Mi zhsb$%$k2>DiA01kIIJ)OK^FDvo75+d`c6=6L(}aN;Cw=2LS5I$w9DKJD}HxB-3lKh z2!J}tJZ>T&s$*CTZ(T=+rICh<31T4ab(5^62w&9?>r!ROWuJ{|HLiW?TzI#f_Bytg z@&23)=FlM)4MYn7SjcSAFQQhnJCRvspW^lthgDbyKLq>DS4E1Dizr@6<)*^pJP~Q$ zJU{C8gGKiYY+1GV-GM*U0p5UdH?5>iB2|SgHa{)o*l5Sj+ZUlfv(m zlM|E-`O+v95_zZl=rTOFOj??O;1w+N5&X;|6J*n2Q|jO&`349HKY z(0xbmL6j+IJ64ku0~8)L2pV$%UDOX)56-W`0EV*k6d_H$bK}_!)!8(TF8~rVP1;*} zUf!Dxe`N~|EQ!Y2eZ@IlzmCjma7d@)6vURa5Cy$^QEDSeSmPQ1O+~$CQ)6{Zs~odt z&)rlh!%E$9W(Hfp=nhs`zR#g^-<+2LOmB}4M{l}B%8g=uRlt2R4L!f5mGG6T9BS5p zq*?c$F#!C{=Jz)%o#s$f6A`W#&ay{uruxt_OuA(~Fe^ab&7u`ILh;UwIP}2!jq|rW z?)_^tZ+6fu6u!NlqG~lt({q_MjYn zWvdQAs2s#20a(iP&r_A_BkUQTvOueetd2MT?kxUi`>Q4Ne%I+``Hre{*nZwd-#+gk%i!j0ZxpDUO+x8*OM)i5?E5Ny9jYwHa5|A(e5{V(SZxgn+bpRKymLyFi z)#YlOD8`nKbx5FoR4KVVarj#hgLg^B@>uQYF3k2hxZVyYAaBO~q?3ui1thec=OAX_ zBIdkS{NVQ$NiF-9AU%n)K4Xhd$&y~xT>BG^el8abo@4vZ##N;N>V&pPR>7T5E#&d29% zVRF5|Yq=TTj}Y#z17Q9e(9#li!~&R4yw_5f5X#ewT=8tb{Fvt+f*M3f|0YwV9Cz%) za1sPmUQd^=hRA#8O&z(;v54da3*&q(5ZSHL5c~;F5ldxQGZp5@`f#@;?QJ#-@%`hi zIWeo5hX`0d@t+Je@LUU^JjO){EX-S1h`4oytOlcv0AExlY&&KhK_@yO_G_v9PU1-Y z`_n>Jaa;!RA9Iu`z5yO0x(^!4cSqY~qXQHHWX8pJT~_~GRtIwhG==z|+m6=_Z?Apj zUMr7VuY+ylLeE7hN?NMj<_?`PE-K!q_t&AxTlsa8SqL-^c2bqN#N?UVC-pn5Nkt@? zM{C~azDLr4_nPg9vcWi#zkInd(4**cggLC$+TWPR^2;z&_*$F=Q1qggK7JjO*!qwu zbNEf?+NHh|r+6ypy*4%P4&*4i&`~*ZQKddqYUz#ZF)At01yU?r9*G- z+tLUFu<#q?wy_^BbNt!c=quBs%3sue?T{O;1sv6HfN|j0#1|W zu=xyu(B5;@6D@?T_wlrdWK-HuQae7?7qC5+1B=hA(qO6{niAzh?!iSB+as^y|L$W- zqH?vIn>@ElNF`ucZXsePJ8x!_?^@HX$;L30aHNR4l~LLlJ69|bn9A4#s^{avRg8S8 zhC&;K?s$jM?900!I*lenPoC@4Z-%y^Hk9YW$U@J4Z4Eosl`i`|W_s*0tLX(|hRYm9 zy06q&OXWFFT&DYuCy0f~RklS$+1Waq@i`-^8{MA{l|tCStk$rsC`~N2@v%qm(%feS zH9wpH;bjgpvCZyFT>!3+A$*~d#D4<-Xk4WZx5nO_3>D|<4jS=R%=^l8_j{IU|NXhN zpgab<6_)X5^z!-@4$xZfD#1lwkWF}dY0$=>xli@l8W@16ZfSXOmm2MBbUWRBH5WdE zd+6*`q=(2p6Szo_|FbfBdo{e@zWuwTH7kE}&&b-z8 zyoi?%_Ps-bZ%UTi*`1?Q2a0i0q`Wy`DrX3hMqj7+U^L%(*Gchsjp^&Xp8JpM_gWE- ziAvwO=c3{JSAi3_!rX3T6ozvo3~9-}$sF^-{u_?}mAKb&yo;2$|M%oTi2nHoUu=|g z&7o-~OOmfrRF6DY=<))UoJNly`FiIZWqc-rI9XO1qLa?oSQ;#hr8GyN9%t0!3;mg; zu(b4DuHrWzooXn+>2pYKbPMq71BdcD<+qJKsmDek{@IQf8Pc`h+P9qV8nLvGp&_6O z6kqCopdTt%$xOizlQgto0*Ug7o$g*eF;es^LCH4p0nPfL zNVuNrQXk7Knk|C5e)0zx3wh%qzL0?PPRlPp3;;N1y0{=X%ZQvXx+Oh;zxbws*Y!Wy zop0GQRAfJs-tN3V;g;>JlZAFZDbo~7oa2+ z{Q;niDE=)neVrm8b$-YX#;gAPVP1vAmqvhNbd2!3eE5pQG`-x=GmGc4Oq|+V-|vVi z#ssCfbWeeyE-vE@yOnTr{vkhj=Y~{)PlIC;(AA6e9qt+9u z&>S~@AF-l2eGFhmdCq@!=5#;*>Ik#@4&c!%Xw6%dIsq@J%J+V+Jb3Kp@KJo4ORgYI zga?|E5IV?n<6)fcoHH?}V7ccpuKl_V^@n?o?NbSo>|?Qx`_p~Pn8}2}0tAr>jcQYM zg8?NPNi1{RcK)#dZLVn8TV?kkqhXfJ0Jw4)dzrK;YWtzjDaoc&CwJbX(ehF+k6Hu7 zz)oJeo0GXzZ0NdPF)m;kHzC9@J;zOu4~k{G{wf z;ij+HqzK>Ew7!F0YQJ7{!kM{QF9 zXlG>jY~MDp`czRE6Y9T=Gej{TKjMp}_&3h~Rs+fGZox`a{m*eeiTV66Wh?}Z!pzWH z^b!5{t}MRamPPb@ZjrkJvuObklRf;-Hg93H4&~x|h4uS?JU-QWyt>^FmhmB?Fmdwi zwM^iJk501j0MPsjVBDTSNcBZ-jYA;n-`x44&Xa~hjRq}A5X;~@@&3K8AE1Pjk!DaP zmMRpC{}Vt}Qn@A+`#|*d4)BSV4L$_d3N*4jPlh=VVbS1xGz3@4v!LeS`x)JRAb;>k zm+e*_$*3aZ6{0$oHrY~({_MeLH_uy+Lu@h+oED}(FOS!K2NeF2k)Y8Y9BWG`G&lih zIGKa~&TP!T-7&j1Bclm39z!Zr?Jw@&-(9$sI4k8!)z?UBR2UU^HZa4k!ps)Hr{*Il z7r=1y$m332abA753b@JnM3cxE8Qn_jED%O`r>S^G9A0vsa^C zYFy#AVTRBSXI`E?Z2sWiwv>NRbF&1=E)FC8ztJgGWJ6I8xxV#hCRN-hpY2KMss~K{ z0B`{c&HK`wK=a~bmA=Qa5N8LNKzbTs(+=#}+tn3&x@D#zXQE!20j(jv#-Pqq6oREk zJoS4H$U9m7{8}sU@n2q`sMMpwKq!zGS6@S@Wqs`G=_(LzB~{D;td5<^&-2=|8xhsc z-`I|v9WQV21P_`UJ?%kNc}mZYz`jot!j*Z@iuh_sC&~Qp;7Gc^HUx5u&qc!XHLHZq zD`Q`7n?{+y-gVWBe|P7x3Wr*XaD?lF*YpzZ8eow)D6*;mwe!@VUh=k6zkHdZ9l)O| zpr>CAi0(VKndG4e!xPaa0f6hmsBL{N0#rfx2z!&WfO^y`F(gQHoeCMuQ>8UFdOH=> zd?Di~@#zjH$Sl~s=Zap>o)&F?0jt04ddpaGG%hu%&K zW3V>KssbuJf?v%M6~ERcYXeB8Yflu27eBpxMmO9e2dsSqfPOAtf8UkB9h+V3IM;hE zU+qki)K3nV#laBR=jJ98V@mX)vu-h-EEJ)n5}Vj~PA!EtBM*RVYFdqyWYMwRl*l?| z;K}#W`r}owsd2>EP+@X|+j#S}ySVj`54J7>`zl$6$eN-u%4gL57!&=8R=H>Cv?4Yt zFLYm3&iz5?lVh2V=kgGY@KI8F>z})Y^sTx!uLA(=YGV?5Dsb(m_-1!)5JVyVX@Tbv z+9hOp2}>bJj<^66gy@-#=n66I?Yi~ObJXt8Ja-XZz*TH3tQm;o0Yi)$C^45Mi*EhW z){xRNO*<<>1ipz^-Y}L4-J{s}^&HqB4dfEue3(|xOV1#zPT0AHKi!C=D`E5$AE7Xz+E>-un&G6Q*Jpn=(!T`}7rb=Wr`j zJbD~o_uyRe_5Xo_T&G=W_C+>2kw1(F8|<@C+N!bSF=A)2`P`m%Q^;oDI+<=lg z(kR%>r4Kc7(0=Xr<8Js(2rintoyD8=NhH>0ueRpM>CNGG5mG#6=ts&sf z`lTtoEo+DQq?_WTrv-x%6`oGf=%=^kZP`>4c?mA{;c1?KU1s?yqFJOEkOHp2r~Yl? zE1nn)(`T+cjD46I&@dW|)TQsN2>iY#JGE zor>ra4z6Km^jmv=i5T;ppjaO#LuxMb_4kRWz=P#5@!F1wPaQeV2HLkM72d(?X8c(l zDaazkmTB|Fo%_i#9p4O$rb^0)&s`n$jonk<0LqP%!pOr{)&_AIzy1*W;7xKXkL}Oq zXH{;KCrBRrS5+MjL}?Z0 zqrIjA4fyoE8LB^`jY4;aHDL3Fi0l>JD$mm#)o?Zb=Nc2M(#Y@%0~o{run3}kv)EUG zZvJQ_KL+5EEOQ*|XD#R-woR%N8f}n)z>{a6AgU;W~cOnfPiARq+2(JL5v|H#Vw&m+}7|fcVa3JV@WZDXc;Vj*M+Fj2~hS z{{P~Xq?b~N0E1jA5@EFTKiOr(q7O#v>$+^n0@f`S!C?08Y26N5#zj9Zaz#K;qW^~4 zy>~|XI7Br7L`CXutp95OfrN3%#_PC=VFd&%%B$Fr6NPVy_0(SB0K5?66+EIts1<)G zgXhv4Ks!hVbogTGT^+z%A!MQ82Lwlsp-r#+`)IirMKlR~^brao5@w8^NiIV12KJ}~ zBHCp2p-2_6wDqp*vnjR%_U&i#OHEDBm>tZ}$Xi!a1dHbM1@ulZPt^n=?&XP5#Rjxo z1|>p`cn{SVC9OP_nEKorAVq?jfDqDi12F>9otDdqWP<cKlGav?B!uw1QhgcSDw{Hzao;P)CRir^?;Byt=hq05Af zm!epj`(_axK+{6c!v86P#&O=gb7Ibm@1U}v2Bh%F5Wf^d@_%8TVO>zkX%sVbC&8i( z9xUGD^h05<5v3|%({b!KU&^HxvZ9oLm|+uCL`1UeXeHbQw^qh8dd0qRoz=<b7DC*09Q(0Y(L4MI(17Ya5OKfixz*#W7>n^XcGKdh^FW>XeMOrmn}jDLF| zL3D_RQgUXiG>T>-K^}LVX3gL~KalS6UDV`9k$&wb?T;KI7YP9bQKlxjV|o{8a(qd3 z8@(6#^@uLD`#+=Qn`x{DWYy3MR(FOwp8Xd&ni2jAp0fz`D~=VQHJaacn~A>x?s%ql zKB7WR>bDaD&9C_J{mZ##gq4%jju7To*3?Td2T`;;<>6feE?!!&J46JrwMWdX=z6ar z(hpFh$bm`UAo=xF=Xj~Y{V&d^Fl%f<5!b84Uful=iJ-6h#5cm1h^JQ}#OoN7`%X;x zFdAxF>Oip3Ql1Cof2_pgaj&0e5cw^Lauj5v2a?RwE!$}Tqz!@x5mn}7iY1tX#NwBi z{}v-w8Nb7i(BR#25I6mKy+Vjwi0Q=>A(Ov*)3(VU$RR(AhSNlzRA!Wo-WoLiTBvCd=6g{eLVe>b=;1eun!#AEX z@6bnH@r1L_qbeu(80G412o=jh(1CIZMOb_-t7j0>fQxo^SjK}tNqK5%m%p}0v5$q> zLygJdN?QG&tzw7sZMwF#FQv;MK%oFo{s}Y5UrPe72dbr6P^k2Oar4nX@p4C`8Ql@s z)+9SwUv&^2}$z;d53NPV}a9`J4y%e>Bw%|qhTo{-9S#&^=ynf088Jcd&O zDk=yfO(Tfpj0JEa9EK_!PYRH#pE|FfTGgHp0aFVt&e4{Ip{8UO2tZG+@Vxd189S!t zRdqY$h3j~ll;npuxkFySL5|(%48J})E$3_!Ph#0REhOi&Bv)$7SHqU z7>eO-h^&W!mT4Q3Wp!l}ts|?Uqb~KkLNA!PvN^`mRrAF8M`Y(RUO?ombM9tQ1-FXyySMuWIxf?88r+gd#hO_gUMBg7&G$3@@!zh&> zRMZ7F1!0uyet|?ZvLLe(ed5~X__OlRBcecfPDp;^@4UZ|pD|6>3e3q4(U6)M;+0ko z|I9NF014|5mImRPJdZ;m5Ry_)xHVzeJ1!Wv3Vi{&I0s$2@2b=CTs5ksE8l{FFWfYX z1ETi6TJ}*!tSBKEQofM%Gq^2Cj}QZQ@~rz#a8Nu@&JyibTI6QeHTYMeT!H@0?#|Wh zYGU)}iv_)dnv)&<-i9yexTs{eqfc=BGx|jHVd!!hV4PQg&Bq`HCpRP*+=5~BzpZfE zGf6`bJe>=EFC>6>{GT(=L$(G3&~D(;noH=OR%8_Eia8DToZ-Jzk-?U@6N-*#VkA|3 zCoLzB@cHqxyR=u9ZN-SaK?r5h9IzsNN?T1o$ao3K0v<{S|2_Tw#=!z(_HS*v+jaers-Ns72rU@} zvBcM>ft8yN;i(pAZ4lk^F@7S=Xe~)7%Xi$Cdl5~@1XssE_Ah7j3tjc%zQ)(zkVgsL z>7*pDpHrE7^;miDd)Z!6o!(-(^+$fhDHTGE{Th_?^~z*FVgK_0*t(noa<&OKGYD z3=iD&MU@GVi^&4HI^92kjzDK#0Arg2{Bioo=xN9guL}Zd0sIG%knLD1l+G{EY(Q?n z*y{+$9;Xkt@At!IR)eZid>@=^(j+q^A1c?P8aH4y{du4R6< z?41xFtX;x(mUujg-2 zj@a8x+9FZVwC(2-h@TFjj+0|4Do-4pX>=~@&Pv>6nFu&tt)Rx z=`I)|-KZ(?ZIJn!@srvotMwg*6*4WIf3x`XnJLE!NvfydsjCE@ZUNy(=Q79lkL8L9 z5+Ey-cmb)$IfH3$@tBFSwD^x z1S);;w91u^OMT)kJ@({?2vy$B#vHr5KW_$-m_W+9Z{?s3=Ian)2*zn?gm2AH+&_7V zyW`T9tginky*&G&(x)F!U$Dyrg(UM^XT+RJ7+4oIm&WVsS$~*UNbb(#Jhqbn#vYZe z-NFnqeZmqhlAe=Gnv|p8J1)qo=!|=AsCH*)Gg8Eoi2PwaNJAleTzPY}3)cM+mRHoE z3v3031oeh-;sytgPDMxc{-0mm^bc)Vzb-vl)C5)9?<* zi)9>{xxo>7&wjco9_X|>Q;+qY!`kQ|fCbjd#?n(+!T0W(_Kib^%~g6<=SDqLux`N7 zd`>F4-7uA|q<^T{k-T+SRG9v|#I)yyGhu3_Ly=rpdNiklkwjmwhB>hGPAZ6jG*v_V z$i4R7+C{QzRRpPCc57~a{CqU=FeDP`hgMAP!#T{0DqsqHyQ6seDMb=jK!u%zqCLdP zNT~gm+ZM#~Wn?>Iun!&HoIcv-hFRnIDK^dRfYkFL%Pj{L#`=+_(J!lGp-2}OV7Bie zfSNg9wJNSKR^S0>SQe^|Xlz1@?u;kMY5e$cflktCUgNZ+XVvGsF(R9@UUUdQ8shFK z6^|pYv;P*Usi$9&Rr*QGVJwnLmLKIp7IuY#tOrL3Wj`EM=v&n373U7w-kntW?D<-i zoiOg2kpOv<*A6q&=^+8fPd*AWOf@T~Ml@!xKiAf@q5AF_co*2AfAJL$>TQu@2M}uj z3Ci$c{J<}fpZ0HPz`8^OSeVpU+qt^yKYo&HN@CHO!16)%F7n!l2GzqF~;kulyF-b#=ATS~jjQe~afYOzzGULaw&=CQ?E162Ox&R-nyLXxzsV0OY z6^6H+vKP=SfWl1-KHj8qfJjt*b+!}1?I$9yh=$e|6~k6y`JV05Ye#b;KM-*0FKwr% zr3j?Ob{xPY`7*cx`2sChgyHFKOwAuc1Te`CLFde~Pw_Cx#gjgzurewLmM{7U2qFPG zRQ81Z#=$yt-~5(E2Nn`?hj$2TLRBl9*%E#wkHEhjKJ^E;ifA0cf6G$MhMK1E6NS9O z!`C&>)UClaeTVk<>D7ebLY#wpMwZi#ZBxn*Z=F|0$;V}Km_1}?|zeLr@q{b?prc9<>YQ_@?{c+V&WMSDT*&sX~fX&KGU)h5@_~M*U=<~-3DmBm$9G5qrPqL>X6!ydg7*kEu$P=)c6F zVw^=Gi6H`s>Av!IDj*5IKJjk8!@crJT29JyuOv#RS+KVVg@-5QsE!ifm-UhD$GDg! z`}q)$D`h1f7PE~#gz=B^AvUM3yE>4~OOdHTwwg3tQpMIXYTaE2z0 zE+wfnp{PG>(pen|ScQX0QQu7#jyZ%eWi>Q8qc%+B7K^%y%K8tF}DvVA0doZxsXC8Id;5dzw} z7+F5b>sN3G!^)QPk>$JmQeHQVNjgiLNTQ9EfZR?NVSo)3-M+}~CZI)r2o}83kC(gz zS+i#C_Z1;1CB}0EwBPK?%sNt7qG8qej-b^q$s9aferNy<5BEfm09tywuLNF3hw%R~ zF}u-wuioByd^!k_UA?i_10sl=F$vVPti!p@n{Nd<$~HBGYN zLs@St9B(zY^0!y^L8W3#CW+E0{gV_D;BA;b;~5SdMIKyg!_P&BX^CwBA+fVyEkn#F zwVx_sGd05Hgs!K?;9q;I5RNJld8101+A!=^a?`y6_N&2O&G3mhG72}Q$q*92 zD?94b!gz<>5ZTNLwWpJqFwxSb$k;fC+1iS^;N$Vv$2B=$Alr>P{!@M$>|(gDs`s6G zs+1BVEPU&sXzHA$gRz(0#0jtvkaHv;PiXzrvK=q)BU@Uwc=2rROXSr&x%#Uj7C$gX zL0FVaS|gewJCoyAC$RWW}`o+iNH@o)mSyMB3dM+cf zomDz3@jPGP`D6HvS4t<;E`G4_jA8J6H-_FenZm|PJVIg%FWf&C zVb|n9dNwH-dE+_?dG`gg9VZ9yuU1bWvk^uJhc)`_|NMM7tlIov6E)=Fojeb=iqvDe zhrIX>p~I(D2wCj9xLqsa}W!+9k6L!-W-Wu!C&gsjt`2EhMJj6=|pK%~BDJ98I{UK014`w3V z`%GIY3cp|IbdPt*5%R$%plSR6_`Knp|MBxMaTddP81eNQ6>&M_^3o7fie2P`omg>cQx$brRtqNx#O7Xa{M1=jW}rKt|{CyLHGNZwZQC1k5n| zQd7WrFbNNnFP8=rNO+Cymf*qrnzu4SEb9!L9dLfS;&1BT`fi=XfZ2utcv0?=F(17A z;IoN{IfFlPesG=y7ej7Psl)DZ4OH=kW73U+BHmf(^YJ(qbw{FxVO90I9@aNqIxraO zzX!-+0ez5jfk_ndi}68hVc`E^GGGDCdZI;?83W)M$Fb>w5Ai#fI`nBS8H{lUTeXmv zG7aM0o$SY(A4w}?_Qv;r8+_WV{AiX^@xd(y)p=l{xD}kPTX0s* ze4BOQdnQ>Bqx|lX897?!s{kD=-X5mv^sP$dhi^d}GX*<_bx6x6O>q<5re;aTFJjEK0Pe)9Y*eO*82>0fr~r=@Pz6;4w0Iku`k z&In;(vB}&1*co;q1gFO)rPm(AZSc9T?c7bh>6GLQQBv|#J1MK%5r^lBI>YwLKc>B? zk(pvvsb0^no$WkjZ=VyE5plC}SKj^gI(*sA{<+(Fk7fG=(mbNS@fi{57i``tTRb5Dy<65*1AAdhh5n)$r((*m;a>D7=m+j*m={WH4k zaae=7lt&pM54@u4-|t_ncmDd#^ZiKJw%GXo$HzS0-`9({4XpMIXH*%abln;~xQm^? zn&fcY{+%-yPY3_9Kl_qroN?=fN7>7$iJ99X`_@}wKW8j znGa@vKWU#q;3X*xy(0Lkw-!gjBewToRya=hfT*MySZe0; z%l#OYWXW6RuevJl`I&Xu!6_qS(m2kbX`zWX&a!v?)?lEiWXQ;=^4_UU?6LC<+@g-L zSohPmeshm@p>EuAmb@N3H?vBn27_iOGu~b27SQBD6r^#35cCscp=gjGq#k>#4)kua z$U@j6RK~xErCn}J#E=xn!op2|!(G0`g)C-ioKQ;zJkVI3F+*))jn8i;r1TQVuRnb~ z>YZ;!8Fl|YEeBgFLyQ#s4)Y~O1_rVKSxpXwtON@jyaI9p+7p$r{2dPD6%{q$eJ3A;7m#ZDp2M2^?o`Mp_=e3`}ZCCktJW%f9ttdu+?(VRF3i zXXt*ILfx*cUx8K`60lSW5B?YSw|GGBG{6DUsJEcjI$^6*4b^KzlEnc^A!gw_pkn%# zGe4CXRONE^e%9~zP#49CfP(Jb9q{x5Y_CCT^DH*`O@O9UCqR?Z`E11MWENRX9`et(h&F%sQe`L2 zXdiQ9HW^X*=^^2r6&d!6;z4p8pLZoKlu;vJLCeqMsld7v6hhaK`)_O^)%chlI5h^U zubIm?WHm4RtjKWixC9BVU*>I4OR&^fE%`+=#IfjTavkU+u<#1`srI6gHQZx;J(8f zVZMln9?hFfn*8JPEwucmN@`4x&ze4*{RxeqyTVr2xGgB9QprN!ELW!d1nOeZ#tuj$ z{VoakgMOwV=k{>$O4dxIRgIoxli6$hHcJs|t3b9H_5#$kU*yoT^y{Tbm&%e74xoEv z{h3iG6v_wV46oX^MKKf?o0!@I!$J8clrLwHQt=oxmgfP9!LdNEz&Nz|G(V4hYYRFw z{T|SWs|86-CQiMnw?N{qunW&28EIzV(FIiayJkU$I>l^_ZV?`_Ip{Zf z0HDJA6Q_>dH^3qPgzm^MJA;bhf<^B{AQkj|1K{857xj&{phdja%xEwY*CBMBgGaAQBdDFDP?Yz!X9K~p z?S)JL6W$-53(@Ms-T)EN0tS*DAPy*nk>mi?L6P-K^Y4}gWAt}b;g2n(TJ_#u>Mu?S znxUvdp=Xe#_7%Bus-bsPAuUr9E{vKtXkJl{9p<^nD@!(mz^52&`XqjCn)RhGk3M90 zy%bRO21NIM3mUc~w;6b)U)ZAax2<*;1@^474jgJX@>k2R#wC>xK}|5D?oc2xG+Z{2 zj{TWeChd{lNe%8$`c*d$+?Crap%1zD`m^vSB(0h$lAl!sJ1yu7YgacfcwCiHiQRCR zhKiXoEF`R-->JGMMTGQP4QL%J1_cGbUMlOVf&T8$sR$Tds#-HzKIeoHzr{O7^?L>H z))+7IF$Xn=cN*=wfO^jubd9o)&`c$Pe$l;)OC^4hVZ97mFBMRT2$(J~3)Id9Yb=4> zs_jKNWhhHZptYtSb5IT`V7-XP5fFL4aPz!luvEBfgeEQnDK^&~B?~Sb1MR_Ep6wG7a&55f}+&FC4wcP&>R0)uJ6uKmpYR0f{lS~em%k;SzHU7NMa zPgdvBZe(i^HHoBX$bHmc2L;yh`lZQK$KJJjN3N8}d-zt#Ck^JE^Xfs%hZFh<$>n_d zob#FzGM!l@W4|3pzx)SX0BH9^a~c{3Mq z47x#$N|~a(wITg%EddQJF;_Tb?PPP9d!Z#}{AeA)EqQTka>(u(4^n0yJ+?mT(8cY1 zL52NmM}Pyk_0e zx>?wx*}ZfIUR&`b#~P2x1mJ>e%>;i!u|-6Afzx@!2)Ji;AN`sCWby`CYHfc#bS!T; zqCnl+!)(EaEOdh8jZUSt<)Z=T05PRAJwFcnPbgv;$Or`vr$F9Iee&=lNH+rx30qf* zJ#-;7+oc(o89vQcbWbi-mSk(2{7Y#viKy;rZGj*)YONmVO0W7Qr#-Ie zZtyjoa-+@)cIxr%XPCAJ-N~dY?)MmqCQTMw(eeWPcL`G*3>U?MYq4*svo3uqIPaKK zq)F5Xdq@yB&~ey}Nj;qHYeUDmUcQLPmr1j(aJ}4?Gmf$}t|@M<;V|hY(tUlv@}ULu z^}Ys#elB<;sG!QH3P$i5N`?KNP%Kf%v1QcN0E0@hrk0o%Pemq*o)$K8r-me#6=Sc{ zl2{^WZH1DA71j(Mev=nNB20xk8LD)cmXBBB-tdG>oMGsY3g-%Ne#E|joF?lX3{&9V zVDrPREj?MXZ^Ky0>)3q2m5Q(WNbMG+n-}EUtL~IxnV#d6VD zj(I5v?a$cgWV+bf4=tP>TDEtZw3>zV#%Y32ArT(8;PZ`9Sd#5gkyZjL`5~BDL6EN#WAA(W%PfGDB%JoFW&2f@jm4@zzYw6DLEdw;mExYs$S5gneT1eR3yB zyU9mN4l81>GwzS+x-Ti2Jt%w-4i~ZJftfW-J0xa)Q?U-FrR`7My@}zMPkZ*+>H>XnjRuy~M&xS20cWAOK z@+M11N9d@J+=E;8NRGM8HaPJLFH%?LVnY{I1=)6Q*P^5K z={8b$+;jS1%Ge+?cG2O|}WDw0a*cXorXCNv`jA+9^beB4Nm zLcvi%;bW+BM#A4v@JVPcVUT78pdBs9e4QP&wet0G`F%`W>8?{4sdF_Es1LkT!?{;Z zkzQ23J6MpV$lBwV?RQ*WU^$Fr4=6P6Qi#JD;Z~@2r*!OHFy-&vrji0i8pFQymJk?| zYfW8nz{vZ*hW??~Xq4zI^$INJhdHG#v^?4v_X9!HxngxfqE`4^m?X*@kGvm8Amco4 zW)B;8vS)jTA70q5m;^#jM$X@vn^!n?v7oAdpb zMqw9-Um3`WiWvf$(U5_Zr9bn)@upYsiZ*-H97i5jH#1xq`4a93XL*z0N#ux3gyNjE zQZkwR)>8!*iq0n*3}l^698WkXn{@;#pvcaMBTy-SEe5HW;dBY*lb|GO!>5}NQ~p%! zWki+Iu-Z?*YiT`-5y(CkgO=F>E1NRvt4K*k(A2qP*p`<7(b)JKXafJPSUr5Vi}7+# z(M`0V_}SKAW`7JbB_DVCb}&u2_@rng_Ot>AGq*|?vs5s#U4z8NQcH^}i|e&0w`C>; zwjN)_MQcDi)KwuA5%1?wuWkLa$qWj|*`ltZTY;zGK5AVaQF+AqNU5oUS%k#vCHV_W zvcm6&sg2tB309RULuIz9aFl$^VdmD!FN3g~+n0{)y*t|D8*yHH)XJal54wqgxsWta zWU_hnHj-b8T5q?VPQFsh}lG03j8C!GW?X?gE&A!hRFRA$M zI&D=_X=ckjC_74Ir-eal$|}TN=%m(BjSnS{$o4mGrQb1(RxF@p zDYIjy!q`PzK|r^dHCcV8z}zj=y)B1>fNRk$N||rS>19E`gJu`&qoSa-mfWa=vinG< z1qI}uH;}Y&n25rrdI?3|J)uq>EJ(1Il}~TXvzWD#ouCk$Onyq}WwE@}_d->g819U} z>AXTE=YSn~fgYh%wfd^Pebo|5FYdA1BOhWytM{>~E}twP#*DRGlF-r`O~?s%`jC()0H5*2N8@=g?1!H^km|vAbk!yqQeDk{Q`wRV zdCzft(N054&xKItp6LZ}UiVRYyJ);OyRB>7i&Pv~Cdw#+8F`Xb2u~_9;)gnw%!_X* ztb$4JAi20Sk;7>J4V5Zwn#Y1Q`8Ur03La2FrHG5~Qe!NLfTW#eBJy9cr26ia?j)#l zLTwsy_Wx}RN64!nyqW>D(|KMO`+gX0>#g1q7~Blw#NQ>Ku@LG;mPHr^KP~mOC}adF z+FCB%%5L>qKxmh}WlK>4&TtN3o&GqVlJbVU>UVWDWOA=N(%lkbq(6J20*q;BNz0L{ z6R|I3S~uWU5B=npbrn{dH0ktKQs>o+D>YgGr7e#H{xWj67ihs-r+Lg&uL60r3+~a} zQgQ_*OJ_XyhP`yd-yL{Hkl9QbVt6`y?NevEi~?sV8zt zCFpZEu7sVh(~jb;p4b5B^IN#1Z0lmg?dd-8F2Lp;Z0W8KE{VvN4nZ14GOAF8LcJ`# z+Dwz>+IZv?3)42_jng279u2u%2atkT0aNoB&=p!?3g|12khC(CLDq=?59=!ICS~Cg z&SdL~euZ@aw{?Af?U?KRdw17l3gMcobK)ZB-1q?(_WMhfy-DNXE79m5ko@dbVIxZp zFUUwU>jM(bG>p44dXTIg;Pvx0OfL2hYPW`C;pQ$SzV)kkq(9m<6K|OVu+_T??hjA~S*WcYphW;6iYTU;6$L&gH;X-q z5pt_7jfhRE)3;$&2#iCz#AW6F&F9i!mBn$G&o%~5Sd)7Di-08msRGR>rjU0JJW(Wt z`uRN;hO7%vOo2{n?GjMz+~roGq>})p0U4e*OZ*b2E&PoE-7y2fO*~THsM}pUM+114 zin|}wc$@riX&Q7=-Zj=Qhi*8)H8N;_))s~$rZiwd&AUd4uLo!LAUF27N3bf;2NBce zDavtD3T40hO@S{sRC>+*Q)QY6y-Z_kcCn&yn)NE_in$3>kHmzuYk~h0>sqcHu`6>w zXj`=sU6Q{sw1{)z6^IcnEq^G^fCzxzJ^6FEKfPT8fkOvV&^+&U`-pVbJPtEN`qw%U zUEj#&gR?7xMl1u1fxrFVEiA*4H@a2RC;6)bZX{hS{@ta>Vf2~wBb9}zS-XrFxTY>sWB0QAEe*0!r$=dGEl@$Z6hwnT0?y^&XEq&O&J?fk#9yu2O%UK;GURv!~>S)M3-*(BS9gAV6s)6cq( zZ3Irx%REuQY_(BXn>9W$FQJ)%B_AN290LUM%Sr7pS12INi@ag^?(QPG6@I1-8zrPI#jJMFZtjHt^)9_y&k&v*Tl0#}HTY+@q#>d`N zt?en;m=R>Hp84zgsOy9k!jR`+OUeu{DyGj?cBwUtG^}i*_%ZV0a=#$@`VcdCt;kz} z$%RA;SQ|Qm#&KjThJoA2g+Ttw9LA?a6iAtj`4&U&-@LRmWiK_-8WC|`yh;Awk!~KU zvml0TaR_lqKGx3I1MkX1P4Lo1rQZHXRKBw3r7I;PGsH(4gGH(gDV-qb-1Spj5wsdC zs0rkKmHN+p@|njAARBB1J@L`ebu2Zk`40O;c1`ztZh~Oh8hR}0?>66?7-KJ1{bR0R z{D0Zn+23V#RA>3;&q7{yjUziC+Rq1EXOLyhnM;IL?d3=ndC1nL@f!-Mt`-85X~K6L z0`4%QkS7K?RXgb8t@E#x6MRr-BYoV>()34GvM0bd>+zbV?~rYSjYHk@hD+AnBbZ%e zOpYfjUcmf~2#ha`(!2oQ*CC7a9=JZTesTDSVWDZFfJ{My{9k`ULs;N6z(%4Ypk18E zUEksX##`>E|F~ap6_KzNb?PbSa zdb9wyj7{gnsy#yFFu>AqAV$_!d^ImVk8O06c*%0EEmcRzXkT4#*htd+)jK>nuhAu(k`iJka&nhRm@uEKuWCI*!B* zNxVi65gOj^7o@tH0Lr-R0WuexygXmoe9?hOkQYH4UZqZ#i~SemDU_cXup%$8AVlow zXQ%h7YSi$qQz(3S?{x zmOUQ4yAmMRTQ|fP;Hu(qdI_B$U+x?1K%MXo$n8CJ8U_1O4!6aHBUhjfz_o0aAIoyp zOdxN1Zwa)lp2Vh=U}Psh-&PnZa64$cqgZ$Wuj0c*!_|!fV)Yz2VV5zGON{OIgj(&E zvk-Bb)0b306Zc=--`nUZ%vWG=b`=sUD3$;_r4MhfHp6>YIipzb^}xmT4qPo-_qhnc?1lyMites~Pk zenkqt(*>xk!D0ggL`s4}9?9G#9CqNY`!2}3rlICW$*Ke|8*1jjl~j3U8P2sHEc`EJ z2zT+w*<#U%!6y~R2VkcUDg<_^9-_So_b8k{e3S@@C%4I0CrsfSHrr}YQ(f6HSrnsgt@by+ z;1gDjJ;LPM$prX>qq%cPT#Rczg7ew(t@Zes0Jp;YB16c^z_0Z%1?caemrhd9;NV1x zW<(5ALrn#AAkEe1+ie!b zsLU;}KJiKuP_VGOu~hX!%Hb$KiUxwQNquO(JmM0C_L#U(ZK_jGDqQ&2erZ&T_F6Zq zdI(`{9fBAf`Va}V3tOb}KWIiHMdV#T%)P6(oHsmKTG>wryubB0!>uvbuf&ca<78|H zHjYj+d~ckG3-JV^m?}DM4s+3SOy^m@Z*GvoO~b{C1f_5@LT6*-E>yQ%&Y&*8>44S? zH-rs^6W1+Wr-@x;#ywVQA69^-CmtFp+=nxB~Wp;PG|F$L)Hiuq=oEi5~bMs_C+KyKi7aN5Um z^KYs=pm05OmkDBoD`ymzs}dx=cIVrtCio;q5|7x2&WWuDOa8abyi-guN|{q=?OQ<< z(b?eHGDC74-PZ7YmHf^6*4Q*;fs)4#-*~~~>BUz$f>u$NS zVJ!Hc(-ItIdco!!ih!$*9$7rtL*ROe>Pea244^ z=BD5o%CP@DTbUrOnJn1JQ-tu-Dqd4jCC1(ZbN=FMk|I6XK(1pb$`eR;_`3pgqbMg(|R!1=#AtAbag9o(-z zUo~8s%6`3s3N249g;X6-IR7%heyhY|u%WCGuW5WZ22K$*MQ}MAWAM$jJAf2) zzFLDHtWNJHoWoe4$12sckruaHobWkJD5W%7W~ov~_^&2ld_+)(Dh4&BfCnFh<4Xe1 zr|pmuv-fR%KnmnS9eN!aSl@nfe!blZr3`%;i#*j7HiSf;3O%SJ`I;JS;grh%+%NT8 zGo>fWnPqtI8-n*=+J%#K+fye*!BoybI2n6_r%D=9N;cb?wLmIYk$QD&NxNWN61a&r@LQ9t}~+n-7r09H6yr; z1!zyInS+ZL>jggiIb?5t0Fvd4m81uVRG%9#$~P}&z(o*T=&%^o*rQEwpR#V0_8&MW zVrg*WZw6cSAN^{lLgXewcPA(K^@J}jqpD#OInv@7;(t4a2y!bXutu7UNOgrUqvT^1 z+=k3u2bl*8bSo`%HjpDoBjH{FzDIMmFXKEAnERhRf+V*TDh-MVh(IFQ6*4S*k`H~P zvdlB7aQSB)_BW(#PM@LcrYJmWEs)j+mW2)J&jEr#0s64))N9YL=7u7f8b4&xXUu@S zI{=v-xAm(r_q)hF&NC&=B(ZP4S6>-#ulNbnCq{?_UN6Bv?!w)_#pjXcUtL0_bj^I6 z^pdB4J5*f&?x7%E>(!9dYE@cQUXx`12erHPZI07uwy?jxA_fUn)oW+kweC{?{`Eo6 zG2%morIvcc^uk?8_MVd+3@a@Z+ zR}6|6HQW3&m-RT_p%2(o_EQ_&wg<8g3CvD^&*VjFQ>XIF|^9l~6!H0YSdG_7>kezI)GozcJqNo#7bg9LIk4equdqtvToa_n&Otcq$_x2R?i0 z5jRabgZ64$6!Uy|E3lDZ2#5 zg@dW5p!oSvrsAB$*-%lV>=2ZbK^X;WA@_*wwZxzgdOyzYi%Rq;-y5V9MhAt=en=G*8-u^k)&|V#+O#)_0 zp|cltH6F*l^6K*uB0&4dBlTkERMS3T17>}E=XYj3qSg4Yn3bN$ee&>ediIyE<(KiP!Hz?vC#?#~ z`9q?9utQ!Jox*lhy9Rl=;T)giL4MAbS$PD7EO>g%1-istlzD&lvnIXzsgx4DcoZQF zkXuzAG^;J8by*+Gr7}=tX@4NsV;}4%T5|<5JUm4-6x{Dm2OMDeQqaD{ z#a$#_yRPaEc=l<&V-O);r_((T!kXW|2Uoxg&^rmd4ciU}vL^kIJ`gky%K3BpYn8Gc z4!I@Uj9cUk`0gv@AR>1<3i`lBZ`uYCGqK)KtR5=*5yQCoX^>Ls9H0~pjBb2T>a{8=1xUW5jJj+$H z1EF=Lc>6g+=E`KQ(}2L_2@K(1zRsGJ`ls~R=SA%$NP;*q!f@&a7neRXru*qeA;cTv;D=P<&2Iaclx6{PEskC0uM9D>KY@$69f(9teeAM&C^Emc`S%RQsDR zBb1tv^rC?gzOdcK@|P5^lIy0NS2yXy^Uo;wVTfezA7Zf!RC`5cG-b5pk0~UtgG$DE z$bkA?WB=abd1%?91)@8%K^~fII;O&lEh+6_=JVwye0?46%c%WoU0eY4|L4E!CW2QH^UVb7FL21Ye90kC;6q_=cL- zBl4~sg+XC^AqJQN2Xf{=5%{w0z$BZbvvr97?_}(xmTeA`aWc^d_Ls?*!|kI^q{zb( z-5-CmAU<#phJE>h5kKh-Eit`rjBU{cHR~h@DH-Aw;w;(9#{E zz|*Boja92UW-PiLu>zafGaa$7?gZen%yg%z-OzIgzv-8H=YzL{a1<=k!gME%(~~Zo zOm$X&{zhYP+Uwk1J?Vm+sZ;07EpZ9?GLBR+5x|BF%NF~wR;$W zyZHu0(?@a0#!UXvPe`n2j3&l=gUUQKNAO#tIG!HS7{@p@jG~W_RVrmZN+>zP4=709db00+oMwOjNFD6sAu$Kj1ron=G|#(p!r?Ux z=Y6#@@Cmz;#MA(Rx4`2*Pu_eS&bd&^Wb`t2xF@{JHS?ng;X;)l4x(Eac{uJ1siH&= zj)>^OPpg3oZPG^a3I}bc6J|t)e0Qqs1;&N3`lRx4Djfs-SguR2xbM9LRTaJVxRaLx zoI_hS%k)WN^ck@WML{z#j9?s;>I0D=XKPPJ0e6(WfG`E#A+odvh-=S7#fFe2fL%MI z1#ko%()d>iK{%Er?vV`#T-OW~1)24aes!?wLSo5Q*=nl9Py$Eq9|-yMtN+_v~@W7j=ZKdCi3-(r5zpAbULm zl>HxWqGx(2tWl6t7w%qD0Z zdS0uDC7lm~yN7x5-PZLf8ozYY`g+JZnEt%?$w-{zs(O{8=s5_n@=2~&q81#ityRjHt3IiWVw;ITR8$`DW5Dg zk(`Hwj=L~NCyfVsXeAlMQoX32Tll)I#TIvVRPf_vmo7)p3&AJ)9*LaR!RN|AV^yyd zQEkMn{*Gd`#1`EDu<{^HCkiO5r@Eh5c!d?wio^1chfg$AJUM zFzgDZX-*8s4Tj<|yJIjIxUsC=s|Y()Y8Citjz>p5Zdal=m~)uaQ4~An-gJR;DR#NJ zaW5R6XRVe5D@4|D5&DcQ-Uv`55|{ z%+}HwSW^%@_V`EAVcfdlir$o50rMCp_=xA8DK|LDC33V@xIUfDos@{6IYA1)5mqp# zMLmcGr%Wj`%HR$lvg^>RxPSO+Y{To{f&_~|1-}PQK~AkVq5_r$ zxPujYJ`15&75-?!3n0c!Z57T5ryx5dfFqxPr|F6<^Wg@r$ILcEx0nkzqT);sQlhnj z7fmD`PD(3b$Ka?h!!}Gh*o$r}dsvD?uhMM(&G97Fo&R(^c8WU*59M@gXNwxb9n}+5 zOkje#MXt;XIDGT}S8nPx_qiQi9SedN8C$`v$16H!i>tEoT_T zzK$t4QJUTu+?&<_83`T$tNH%_a#LR*3;hgjE#})pDeV8EIUjs4uBi z@)VFstGTe=vOfIw9`}XlkNy^DB8c73}eh%gy}(K&uFewVS|$hv=iIw8`NyQb6O zsS+Di*sC3b?-3|U?f?)4$|Cz-tGaJ{RvnXc&!DXcB@8+IMcb3^np00nH-~7+DR+}* z-t^C5;R+P+?*pN!bSy92*`A9g(pJN_7S-_x*AFaroSbNRGCTFTL*~q{m(;^2efz8` zW+MH&V?|0_XEaiF*`-q!IS=@+$2>`%;;lw35Ef%{z7NSRfPT4_eui+wKbTATJv?o3 z^#(OJfuiT5`r-8XLsenKj!pCaL+N%cJYouR3ZC_sTkPG|UvzK%Vq!jH>*M`K>BaB5 zsvp`ZTP3w3&!1~8-eDoym?fp?d9&@Zc;-y~;8sa{eu_ZA*@^h|qn+Vj=%*Zxh^)I< zVgK#(tFFJL_vDNDrQ@Lwit6%9T)%P}`kJiIEu8ShzgKf}-N==7bdGlC#<}SP(~0=a z)}2cQUz;BqZaw?0^9#DLIQfc8!CWIwmIOse$j`qZJzA8yFKDX9S-p=*MU!23z2#z? zr4FZikn`6!RWHiRRP$Yz>8}X2TNGYWVk`j~j+Q|E56-l_=r{8M#sE17S)x=}=zw16 z`cM>A!DGJ``*(}BL)SAoHOptCTwNAK4S)V5Jhi#*qA7Z!!)ZZ!Mq|Tx<#1d{nT*u+ z*Zw4-528G6e~-f7OKqfFq*O|MH~U>LV&|t;*3-Bi>MN&~ zFXhIEn#p^S5_3W(zb^4pVVAx^nkG-Uve#&m4x||?H0m1EW@Gl!4^{2(Qs9ZU;FQKu zZW=71K)VID=#fUhmyNNa>m(UxdqT`#5*ohfwG2NXcH!yzs4S;{^^e#i#YI52*>}pj zbBUd`8touq`R`-H!G)D7ETavZcVeOd7u?{tSI#E7HPN^Q8hTY__t};uNx~K`^iN__ zlS*e$6upbe=Mj}hI_R~r-z7S177NCv$f&E zyK1IMi9LHn_Mtu_ur9SlCTMU5JfXPs90Rp=cQxicmA`S=ED-`Tf;1HL!|WlA?ocC{2v{fRdrV7JI+TQHCc{j}fKq zv}lY8a8g8ul7+j~xN*MwDc@}|B?H$GL_{^mdEDq6djHF)q?#q+H5Is{{0H8uW%Hi* z64+-atW{@#&(0fD5>4pq9;7O^Af^5bJ~+e-un4B6}0nA|nrt5#@xn4E8nMuz2=f&OG;yIqx13ct?ZMl|c=i&4!hjMW)!M zz)(GjP5&N*L%4_=y|RNsf5ysa*uG}4mZcz+!2)0WSQ6*I0AJSf0DQ>>wDh&;;)@#b z<=#=*dNq(oNWg>8I!AKCDNfgfmuMxZJaI<%9_;iJ32u4ekJLd61v8f+F z@5n_=UEJUmW)bS^(TPUMsC zWvC`~JiKQmfSOz=Y%MkWwjH9W1NP)QRQ5Wc3X zfxinO*5+{M_B3RA?g3Sz{^(?p0QjK^U-{uXL))VHF(G5VVeW?dUtI@)PgnB{9;8bw zJ};U5IJq(+Z*Yz@b@qBmyU3us`;94syW(vxz!5ehOZ+-GFV;r5H+!jtS}uZw>*oY) zD7Y%h6Au0iQ$z?ww?0v;?cEN#8{msG?8}Ept&s~}bA$YDsiy%UdX7woK!2Y=d6!n$ zBLAzgYF{tw^*wG5z;OMp65T>~bkwO?#zVcY%GlvIJT+fzpqEcxiy@(44ZbUPIw|Qu z?!5s;c`>o;Ppcw%B7sP;iZ_c#scn!BT@e2D9wpWJGN&E{ueB@we9MXUJ!u{t{Nm?c zjxkTsNwv5)AC!5S3#F~74N&m>kQ@G9+d$sBT8ZlCV6<3>GXLdJTkZt&@fnDC6QMz; z9?ENi7uOr_xMS7NAJ1Z_@&Jhj9bSbi=*({Ih%|eA{7x?YwQ`Elt>APnY=)KbtFe-v z>bRcd3&nl-5%?+Ot*)H6lEs_&&c&?Tt1q`u8g#{*_OI=-Sb4dscZ;V#LK1py(6qV+ z((o67jGAD)ST_mP+ot9(1jRdWj_DLc6L8+2_{k?BM|t)-1K|#2!>`iO>t5l@oYYGe z*9g|`{sq=#YpBz-qOsSYO`@%?_zKrh;_HHwdeb*Wtj^F8Jp%y)`XQfbmR zOxnSaDzg6YM92d_K`7AI?6Y&2x+ame4u)`CRzEJ(&d^x0LWak!-#aZs5n45mE9a7H zTAUgfOL4dTrQ$ylayTiuIsV#@h0#spxe$)A1*m&h>7itQRP^N(pCk{k#LAXq>PVbN z>Py9-<|~;%d>J`DRhfcGp*8ybfwD9km{qhgf(XE&CygRE@vO0m93JPYXk^-AEFV=x z>p=l3n?+E%Wyv4stba|;lQ<1!gn^$1@?RwlinIq_uF4n8DhV$ zfGj=sXP+jk6JwVW*$pzSHXhMAP^1AVW;6t%gbf?4%pANz2S4A|SA<*FE%zH*Oe|Dg zcsFLixb=qc!Gj&}MXz%yu5B~PF1-%U??>BQK!|@V*sL87&-Y7aupeYrR+}NK)bR^M z2iuk6M6F0d*sek!%X%m$T*;JTz=YFZdy49Yf7nlG%V z!`r*x=j(XQ@NA}Mu+Oa|B%7ciAv3hEbOEChIi{c~7?efBaZ;fFJKOw)DhdpXe92U_ zUI3Jqp5phTi;$vtas{+&zFJ3?_bvxy)BU2DW&xT11iYT#D#jpnv_>fK-Ebf^eXM2j zO|j+pJS@g}xlPZ7sz+4Ef51)R4Efo7Sjp$j<5R=KYzmD|P&^;}%%@hSO74^f9VjNC zbB$t7^a^xD`ck)ic<9R{1Ht%-Q;#quhRDF zUH!*onIW<7UoytN*pI&hghTD5PV_a`LW zr3z??4bGwvVDpI4ZwOJ05VRkE9hY)#%F(vi2MD(2)|VDjpPNW_SY8DwtF>Q2?4h}h z!#!;;zp4Yjs10^9qb1o@6X%T8b6wRW~i-G^HwtI(PX#5J=m z*7b(E_l>2Q<%d2dXB#{?CfAMnLtFH_^pr-C;_uIPN1&&T=$L(!g=!gONLe~k)*CBj z4{`n=E)y}a2(c|~bfi8cF$4AFdsjg`b?4@yksQieTAFGwT1nX9ml zF*oTvQ>dl>Tjd%-Q{pBz8=zyUqYD|)205%kpbvm|)5&{oL87n`{=%0-9yTgnu>dsK zQ#&RpXH(;4QN55Ta%pNFX*(s%y>(9~O zWln55NpMwdFg&B}k;ia#pj zSQeGg^nUoOBDFNEL8qWOR-#plq0K@|;-4n}o8+f6cN z>#2CIDck3a#i>=dq=T~PZCbuWjS_{w>w78FrP>#PNj5T%U}&-*iMs!!fOSc7AO#AY zNA$O|nN-P>3)&AjF+u~g&!6w>^QENbAJ#oM9Zed4&_%SH?*VyOs0)?Yqwq^RTc4Hm zDEwN;hDEE=ItV`7pMD#iaozfPo?l&5dK2a?5hc%;ar?d0Q_0y$#gLFzKb6 zk$hGvmT=*+ZEBy2Nz`EMnL-oEXuANi;3AL_l+=J}tfhr+wA#%{zo|iq@1j|xcw*n3 zFOyHd%?W*PA0|IMSdrWxoFU;fX@t4M{=IqQ{1S8nAw11 z?z6JJOb0rGR>k_0)(Kg3gQKBh9E^9$^=V@+Z-vDknJ&MRgNlbLR1QR66mKucOMf)b zTRBfZX)WQ}aN5w5K!|tjSVvg&#!f?y(ZZGbV_73xuI+h-m3m3DkV=eCQ=U4V8^@c@ z*{CXyN7y#lr$&g8c$CCB?tS3^yzPmSCf7%D0+{nhzDD#=&(lU)m$Fl_lf@nmy63l3 z7~H29LTnLeEp{M?t#~q4gzlhNv~#1BKU+Opkg*b-n1rp!z@R5g;tHiYG|!$!IyS}E z+Uym(RTIa?lRw%JimBxKIOg*hD$3W$;D71iaXvSAu> z|Q!~rQ<0wHC7IlKs_PZEAfJ=JWg zDC+xV^cre(jH=|>C(ITdSV9B6o$T$vJb=objLyD=INGf-Ky>sN zUGkCJhApSn7v`jVZA{26lk&3oED&R!N4#z|6=5NYe@WI$FL}N-+k-7j_bG;N3U7og z+e|)e@8JzhEqj)`vX+g*GQ<{ny#V(Sv~hYw$f{dTdtc4f%_m{<@gmQCB{O2Z_u64w z8vId(3`QH!T}Sw*kQ-NUGT~EY7htTj-D9gRnIg>R4SH~al_n$d#m&JN+qQI zxSkvCENL1e-I3i=$J&KCy4_N-wW}r*Oz6gWB4yXZCg zms;Rb_ARcT8re8}B^U9o%K(5rHYsm*vnAyjH5YY(3D&P5O4 z*v2bUnbLD#Vl}F+i!Y1cID<2oajb0=^@aGjpVDxCK)uLY;93)&al{#3vW%Cc6nOYa z)u-lvt3R*84DP2lO4U>^((9Ic#~NU((Xal8%ZNB9k<-UzIEodq9UuHpMQrT_FVzk! zn*IkOHrTotxL4hk0GPmHoUl^^zj#-wmi0KX#UJ5oxLUX!=)e96U)zVZ^+}of-?V`@ z{gjGOtiP{KSU9CqKSQ>Y9!g1&i2Wl~En_OWgY?2JcaRoen#`OGw0{9uvHiacdmv9* zI5k8>AS8p*JOqCgh>dgiGa*QV+%@v_Y2D?uuEDOV9H#Xb>!_V2$`_!zE~tjrN}jDR zLv&4SRWbx}G}=*#Uh%*BdvR$Nd(ht-GGLLpPV99T4qgJWR~}UM9PUr4@v(rtML!g$ zy}u>QH$l=ucp%T<9(-T=;#px@FU^fz0x_b|-1qj&6p{U(>T3A;{%`9cFNi$?g5|O=w4*0*v1E4n>jF8zy z@hjg@SB3iDNwVN1@Yy{eT)Q?MDMJ(S&3cTn{lAcCE3urK9iZTS^wk%V0Q+po+9BB>g3?bk`Le+cOb{f{ z#K{JiLK~Fq2u2xwNHg&WovCg)&%*wfqhNQoz?s@%YXvic*wGG@VJ8=rW+VeQgmVd? z%y|iVK+X-qR@mKY|N8OuJlfZD-hwAw4rBkY2UtVS_9=5xf!I+9iML$AMCUvB&A(cj zY<_6@?DDPZuz$y+UGPK(9aeUsvF#8_&XOAuYee#S?d&2ZUsC7}BE&=k3dmgvd9{n> zl%mp0@`~W?3XatsKtvVqwSGjS`VF!u37{-J4ko&5l^2ewl5<|8we*U8;o=7j(dpIu z{@-8qKlOIGe+)wfUQ7a@(Of&&?8udyRVGZK$&)!!bMfpAFQy+q=7$Y_jMyDk)VqLD zJuLfS&(V)DYFm|&2dTo2#TjnzPS%D}A`D1k6tpMzlW70c6eNkC@Li1m9%QInGmmUF zVqqd1Jp1cjv|-EBM$ew3?d_-@OfPDEQ)$itLj@v6SEv4*P*&j+bh+tvbM)E4KoO*} z9#ATN_D#+S)Nd@IX>#6=6u`4x?uM>*>fR=lsF)e7z@sR?xJ$*E@6G;0X~9$_L;rIq zxxk@fo}LlI)&d}^a9)~6*hOp8psdlLK=N+DB;M9MQ0>zs?_kAg(p>E&!(ho|L?&Am zGxvN%VAix9&H6n~*xF**i#&(wz&ytTz{uIPK)!79F#GN-{wq(r1%)-K%7)0LnLWG2 z${`k_msm5E)1x*-k(lx=nkhdxWUB-U8)sh>$aa}EjiUD)&rcy*K5pn1lJ z|M5P6^WvpDwVvDZK`@!*MT&o?OhRss2Wez)i3_TdSKIwu z1)G@}wAo|ySPEfYm}*;LtWDk*)O2NcG_H@)38B%bf@)k$=9@K^m+|Y#w_+2Oc``$S z9YJ;XRZbuFFl!RTWuEI$j2y@!&yMgpAki|swXwpjt4jXtiI^WDdAN~*0vD4lZS=23 z;R~+*ljOpG1<=JmG+fZ!uaD3n56M`G!_g&;jHi~CezeF`29>dqMfe1gU{o}tS&Ec= zR`s(nkxh;lG{&hARQK1GuHnMTInC}{?)|HzP_ijQWOGYo*q`IR>BeXcJSDBF-M?Ur&qW({QJto3R~ z=~V9=SKEEU1E%7CG*BRntZe!7fJE^T16v0;(RUaY^QS%vfiKn}#Z|+N;TONTA(MVE zkx(rbe2xE1JGuqd>A-{pVeNNSbiVFlIHeGFcEThj1i(-->m)SYN(A-)E6~VpyA&uP zJvy3FKD_ZmS${xoIrv|Kw){NkYQF=(GJEQq_qVs^po2=I+W(dC=Q6BlHS@*KI%3$>a$x^3<(s|=nr%!pix3K$5x|9ZSSn8;E7&(0K`((p zmRsPaomuHRYMfUFv)!yD&aFr5oYd;o{t6c=q z-{uyy+FR*S{Asbnave3ex4O%trx4Nx15zhX$$hFNM-+7*`3dDDUdw6iU8W{4ol~w- z4-2MY<(&=-xe2#u6znHU-5Hz(@-Bp&b2X(E?Q`If&YYD|GTgx;YMH#S zct4-`XBfn+ff3aaw@G}x$kKx47lKCPU9NU2j6?zfqIJ_Kat zR=J021`)M%Ya)%BSicuVzA1jvn2wB)*&!Cdbvqgb3w;#;8{G5_CG+*+L~~Ef zg1cma7SO4tm>CfF{-MA;UCxhcvE>qU&A+ywf{ox!k~>{+Ub?%7{w3OwvMB4%{e^wX z|KNCww5ZNu*ew|8z4QjCZHoZt7zZ&^&f$i~T!#3&6S>(fOd}V3M~BV_a6Lt=ZoJIE z&I~N3FnB^7Het@}fPAtbqKA;7dhydaf$d|TZNp%ul=OwN^~_t(-UQg9mL%oS$=ulu zD#h+Z#%myHy(W^@mzXZ=`NBH=`nEiTEHmi63pHf=zgV#;l4$NDAvmeMO83oJI4KKo zk+qh0^u@;KS4^9cd1pDS9LHgvdUAlUGv`3%HGjLd|O8l=;)m>7WlTJ64b z1$hKcQ4CVYg=;@jI6*sQsI6CcwfbKaKEC4l0 zy5W*7o?iYs7mTbmQ3%JQm%;^f-hI^1$#`IS?~3M^zhmfMF%qN;EPPXb!Q^kZ_r(hD z2VX1eOh_Tpc0dLAA%t`wSocpBKOdF`(Z2^o=QJT0uG@CeY4g~WeGlj-R&gfn>pRK#1N&YR=%z*{RyrR0|skZUnJAO^P?z0AR4vp}K`kJ|(@ z*MSe#rp-F%Tin<&YKlx9f^19?-kr;Ao4>agZ_Ga?^Cx=*?cWSY=b(jEuY{_R z9^+iy8%$A-D1;u>uv=^AhIQ2E51X2o-g%6#n-pHzEZv5gO5^z2+G@*JVGSm|@aU+e z!uU{=V(0;;gS=}PR4bZ~zO$z(JopUWfHf+yA=V)Pxl5-ROuE1d?N)mIG0>-cg#B0o zG8AifhfKoUT2zu>8tzMOvmF9?H?kTjv6bSKEPPuHgJbL4mJUtyi(^Ht1}NpV zdbH^3o0-b~r$bA}4e?d-(v_gm)?&wzl2y@tZigh^6cXI5Ki5Oqj26c9u_fEAFBhv` zwLb7`tNacI3115C_iNh}Kba!FF-Px03;wv@PzyM18`IB=23L9CebDCnbHf2}Do^g? zgAAxEvb}QM;GTe78`oIl`wenJsJGj>ueK#XM*8ZUp6~=Hfj-}d+G4sNX%uvaR;XZxD>(-mr)M)xcqZ1q#m3G^YmKW8K_CsKhS z$9of!JyDmHTjrevo#g$X!(zl0uyXx1yQKN#??6zK^U|w;SxB7HkfeGB%DP=l4O7hb z29DE&Q~%0Is3fezxNog5_e0iV`(-Lof8ZUW`SI7)I@pk$vhvkH$=xR7rTny#`pu3T3WtgjJr1yK z^cM%AselQesA&L;IY7n`aZRx`^X6`CHRuD9Snf%&T+7}=zm*wgCCZoYqnjDWKS0Bf35PdOV@%mgWjNIRsLE?R&MRu z)oeZ4>zb*%i!$b!V!t0nz83z;iDP6jIj|4`FivBP@BBPF-+`k8v|W!h1>Wp^wl=?k zTvQv2?OB%pdV(!G-V@wMWJ-l9s(i|SLaaIh(T9c7@*#d~HLfqVe{8P#PgW%+QvY37 zNu2gYKsM+@{GN%T$Co4I${KO0`*^q#k=@lU>C~D8h~JeTQX)m6+||Sf1X;?sc#ya# zN*W?6F0Abmf1n^Q?B~j<5)j%!{P)cEN#bKb!hJ~xn-u2$_e(Ib93ECU78BV3S-iVn ztE;YJMRrwl*tohHApmZjCvzFE;rl3G7Sn_PDLl022^U!sR?SaGr-Yna;TfB91lVfE zT1E<=u(d4fdI*4rnIE)9hQhOW0A!z+@mdqSVQSj9cun|Y{+XI$*Sh7l+v?|;C|;&wPax@v zlqf!Uv$#MyA#+|_Y-fqW?73eR%H}SeOz@8qIPpkpx-D|;>iw0__EIiGK3pG-*JIWS zFqJqE=*!J=A(xbC!5j7#z62;|3r6HGAAf-P2StJ(se=QwGw5)HiI`N=6WFf7A8NHd z1HDOi4tZEVr61@~&|;ed@qe~eVC4e>ftWqH_>bX|1GpVxEh!tIBu2BmKf((Vrc$dZ z>c=Q}nVGFBaP2@UEnRdUK(gW^s*Uj1#_Ea=5ug|twWNlZ>5B&<9*54}xCZI!>xG@& zO}BAZP4`VM&l^s|3GHN^$|t!gUTU!oR8VhpwM4x1$tJ}qy~RzLDv;7!C;+)6f&cq# zKTpS?7TaOKDtsGkUzdZv8zECCuXSPBWi2*gxQc1xyMmg+@R3@=TH=t5vy7ls)f^Bc$AdzY!nmFwX6m;^ z1>h+$qNkJ^Cpw&uI=MJ^w~k-H>dhzI?mXlSFvISq5;M&BT;M#4GS4?@;DCnkd@>9~ z4sb2`x^Q9%=8Fkt&cv<93%wJQ2{`ok96s8%H_A8W<`Kk;yp(tk6E(t$Tw8oW_`nvA zEj*o9aWZuTG5?)vXMr(MtjaYjm5RgwzmE$jLCjr(qRY?$r;pnAQ!1Dc0ojujDZ2>E zd{CV7aIxiiaJJ5@jKrMS4{H35{LlFbxE?sCk7PW@jffujZn+YC216b=iSzi~uJB$S z*KAl(Kz0|i6z7g}#qh$h3ISq1<~WSze1RNthA=#i#c2AE>=Pz3WDp)#7+_Kb=X{zJs89i7RUS|2XL#kvyNz|)1(xl@+wgvGvxaj&p{`_9GWVcvoLR;QM&|EHpBG2nio!=mKq%N+oVs)Z{HMdaQipB6O-E z5{I_|&P=(L{6h%zzVdCQNm4DdeVo1zsB!d~0SsuH`H@b<8BGNz?JD8!f`+l-C~$tC zPbly00Mw;SHzOHrz|gn&BWx&7*gS3lK3=ghX^FlA{+I!~HYKO!!=4L&Z+E&(kOuqt zn~HEbKSso7E(HOb93K8QLq>0UW!_u&SwFg*IFIlXnAFvQBzDwCnJ2|7nXbdFgJ<{^9F zu)WH*WPi?43yx0G3Vbn*NV=Y?&T`xnGXsLsao-WjgY0N1oFvwR_zJFwPzF7~;lo8o tnBhPA%ivYO&7rFBW2yovt?@gDm-dgIoUwW?xd;By*3efkR<*wMe*l6CC=37q literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png new file mode 100644 index 0000000000000000000000000000000000000000..32edfd280a9607c008eccde31c02bcb46e3a2eff GIT binary patch literal 74751 zcmbrlby$>P_df`T(lC@rcL+##hja_l-QArcjdXXYlypl=Nq4t^G)NCI#6AP>H?H6A zwfo18&OI~FUFV!T&gTRX%8F8G$OOnRFfeE`((gaOz`PWLfq{FE_zHM3)vKQa1A{7J zB`&TkBQ8#+?CfA}WorflBOQ^ffuN~6@H$5)Ax^{+7XEwiDttVFh%Dl}PZvz|(rpGv+%>6d+p#7}vroHKWEnxI+XSj*| zY86)Gh)0TQP+|tQANzH<$>;k$nSr;ild2-{w<3hQun6ef20c-c$Ou)a^#RPs6))Z| z>Q|>)-`75NyhbAt&P9R2dg;W+oSTVoDhhLAT_LRt1EVsSsqhXY!&)XHhx#JeGatud z(5W7$Xy{ZwT|pNWWy2jNNsWidi3a9JDwz`VUvn|E8UJwN5P?7t3@2ZYke64Y7pO?o z7R~NGNbVAUPk7h#L4n;*GqHPs==WJuf8>WxWClq?=??*gn*phxoL7!-A!`>8_%7|@jfs0|eb6>i=awT*B>jc+D#`TFu zI*kys;?ER2yr^yXIJBE|>I8Y1IN%4$-%h5H8bjE%}!LDU>NU$+ykS z0}&SV7X}um3&?)*^>hqOn8wIG+aaHW5yL-{yZ4VppsCJBy&E)^KwSu@W}UnqA&*%{ z3A*39)ies_p_4Q+NJ$muFtAlk`9z%m3xZO!A@=vCbFqm?>&}sBp1M5_U-*0Xm=ZE+ z21+a`h%T2O9K{?J`*GJ7gnQlB-?xFIVxQDS!o9WgO3q6)T<;9U1lQy=815(8)~#n~ zF&Vug39Ww82vT%HUl8+)^-nM1V2Gs+6%Z0dxj?R9PH-o`mlEg?3~N8UuVJNOs3g^( zckl@TO60Fx(5dLd2~U6)GbxVP!vPCJ$6^qwmP5Yndvs4YClbULO(*&kgw#af6c`e5 z$rFrYBF!e2Rb9=3`^Owo;|_ND{$PeT194K#i*eMfac@LW$zNgGIr=RlTy=G?&tRz2 zW(E47j%>CFt|R4tIf-+O9#{CuDWlBz@OIECk8mI1t#kZ?36rB-@m?fkb5)s`ev|K~ zP*tB^9x2T~6u}m;l!9=b4BT6##emFC3?*?RDR-)Ge=$vkYc9tqKiYlN3;X!J>48=F zOmiyie1zcN&4Tbj*x|V$A-I;h0mT0)yHn9{r;1XOGS8%36MHt0IC*1z=2EkvIkdI6 ze`6q!?{mO4p?A%s{5JnAFPP6i{tEY`NM>!~6Q>r39%KJ7eIENdUZ9c%N)!6GUBwv1 zQ5fkC=CirEOv?q27ZZEeqa%+mV2)=yJ3G7An#R>(z)jspYj@eT=D(D}*kCL&o3Nbn zv$_lL;C$tZJF^QBqCVkQg`PXX+5bYh4io_)Q-~5tzrr@U&~nWF(y7LRJ^Z2|kBIR_ z=`X%2Oj}s~U-ng`9G@xw!1JIneUke_@biU+5{4)2Z72T;z9) zpBgtcP7pkWAEHI5-+&GgbKtqiBjRMzWa+5Yuo^JYF%>)8ro}y_sIc6^)!%#k#7m4= z=)v#l?YZm`?Ah#L+MwKkiFiLrxe(tf6HkR+@<~lrTW(Rpnp%(MGERu@EdxFS@7tGe z3+RxNBa`Wq-RX%{o69cL_Sm(uZAHOUQVCvv_x=|AE&u!aOu`xChFv1aSmLy>N2TWN zIh_sNWs+)AOA`0s;9%h($>78w+*Z?|P0|&8?nnQ;I~57lUiFZIgdY?tT3>7Qwtf!2 zOXgOipGnx4QpkU;oT_Y7LR)fGLaJu<1O12X$DDnl8Ty%5GebX^XZo#QTg%r+SW8$Z zTO&*}{kSYioOw5+wNG@pu>Wy?WS@H)fy0u(goA?fjN`S<;WSUtseIyhs}lU;fhop` z<*!~J&N0!xrB7&bFxd&&F&yEPOMNLeE9jjaoTZ;zET1WvsVpx!`kI|5sL&z{DRyr} zFElCXo*kIc`cZ4J(>z*)8A_Q(c}@8`x|r0-tH)<0zS+OocZF?5uzAi^cyHno?$Y@p z<$!a4WvcJe?(*AZ(Z0jDaKRe)2b=G6(*OEv;vrJ2Tq0<;XvV1KZ){n2S>CX4u%U9{ z;8$=8Sj}5}H~DLkWLG!2o2F99Ix^pScln$)idhTFEa@bVv2A@Ghl**p7h8Siv?Ec|hvO5Wxhbqln~Jka3w z;I7tb)2VD}^2zY2hoFVNri`pkbD9nMvwqClw;A3irNli3H+G*B;I-hh<#}}B%{6Py z7;rdS9owqV#=Dx?*5bD-WF?g8+w7;{%iGS`j&VPH=jp$6m&0b4wlJ(Woca*+VEicl zaIBS6rd75Ht103rq9tOHXKv{Gr}-B|v`{o%)J^ohv!avzb8Hw$tV=8F{>{!JQnLX*QWYKXc zX$`kr#Rfz$yDuaZ~qxV<4oa9 z)(O`^ZWLI^sJa|>h+4C5+;d=91G}|2`dr1N5g*n6tFPxrb7v|SKcI4JYx&@zw@6&o zQnK3ae@>DuxaHHSKd`z3nFQ;tkv4Q3!EB1$i+%~339}3vB!~;$L%WaGie6C|P|zF6 zAF+F<|1J!jpJ>&;=dbjP%F#EuSe2M$B0}4D*2Fwae5)Q5=RMyVWb8Z?X|h`vNmf?9 z=ntnCrbLUBa*T4U1P}bc{k&r?BRlvzgv>wnEO#A`crIEFj>Gs?_z0Pr4Q!kLI+YaI z*v?fCwWliSBK>r}rHAZ?*kH~zSM(Wlcu`(TUMZ}OSTs62v^DE_Le8_BJdiww{*|ni zwmLX=Y-3~!gI!zPNc})3d7DOu-TbjXWAz0_1MUin{vutMn|o!5(J1`|)a5=-R{O#iur%^;4An(gHY;#7nZ#7kQ(j z-EO;OH2fN}D401fxFmcP#^ewl3;V~ignRB?P`>|Q`m+NzfZy;9&Vb`u0!)fXGff$D z1qB#-;5#A=JS+jsOW+$U@D_q4{O|9Quyimlo}RBJc@)#Q|?fAVxbU1CL<#ga5gpP`|w`! zKiz>}f)tjnu8w@n%pM*dOdjk^4$c|KpK-`TrRJ`eI= zF>$tXbhUD@Cxeb_WbEMPDo8;Ao#?+`&vcr3TK&&V_AdWn3t*5LTEooB#KQdF zV*_0UpilXftvtkn-~KlE9{!ciu=7e#=9QUBTV!`ntYHDyc`3sKv75zzSk82drX=ys}Y!U|R9 zir@Bm;EDlT`dRjZmqm`el?IFAuJk8(g3rGmAFeunxsLMgWKA}??EHeXio#KeK>uG- zh(JCvpGBp@0}Z9~Uc=QH)Q@l0QpG939nl4E*BYkLW1Ucnh#yLY%t2dxf*&FTLf z>`AiW02u_;(`n{d#Zbb{gj(g`nkYoXvr0D%jh`u@z{}=lH-l5iWq5+Vx@0=Ozi%fVdfzLM5Llk3$2NlWQ8+vndP*8s@VPJ#h&SaG8+@N`LtzEm5HPJI&o;=PB$w(Bg%} zFe(=|u(sy(3fm#j&8!^;5O9#h=^_hdVC4NLaA~@1q8E7}1+W*&kf_E*0AdJ>tJ2h- zr^f;q#6tEbf1=(JN(`EsraV2;dnyrf4hlMN%0K~79{H&FTNbqE&E^D<-%Bm|`25R@ z3=mh?2SWmBGA+0W4b^)sn_&zphLl(@ach_d>%r87t_UhYq4 zCo2c03ib=T^jn{a&COJ1CSv}&L^ljiokIC{`j9cRFET|jQvIGp z<}mSDxTx~j+KB)j=(0#RyYG=@5(|m>XRwS2`8?hPnWm)k^}m*iiX-)ZtKV|f8aQdU z9aFOvPib~Hrdr89a8OiR8fmCXj^-0umnzvKO4c22=K_^RNmg5 zRcYXN|L!Z;?s*i$ju+3{;*mh!*G6mLC*1qnX=(3EzizEt0`bE`PSkya=h5uc>-Z*% zV=DgalXl}?n0-2H+Kgc^_fceWalHiZHL~Ikh2b_UK{S5)~bt~ z(%M6=!x#AM?lpek(8Sj%LM|RDf{HgHLN_&@N6pxHDedR3DS}GG)T9Bd1Gg>U;D?%D zt?d1)MbQZpxpXDVY#@KdVb4dHOj~MW*zLpNf^KHtaIDW=0i-^ zh1dtjYlV`ogx~46Y@~&l{<>oygtdrGiO-d3K9s2^q+H z+0x&+6O-a0M6cg+9sbg++Mj)Zd?$d)A%Le-$Y)|5WpD#!gepw$B)*mNhv(7f?$>eD zkTJ$EgX`+nn@;B0wVN;|GP!pGwM1SMeP_3jUYi|~)=!MJ{lm`F)VcjOlmQJ7qrAnH zjQmxlL@s-@XM^Dz^|k1S3_}?E^nLo^h}Jv%)5UazVP*zKzAr=pH$`U~{K5+sN^1fB zT2`#OkRDxzTYf)F11+o53g@om+(DG?FAs_<4*ptZ5n6wHN`;bl*68*5&DL(rW~eRV z6DuFoa{OvN?H=DX_t&6!BryETq^oQVvXz zz0vklcCW*NQ}Dt!k3H&x@@c)%rWhFlFU zVu4Iu=RQ2=wW(ek1Uv3n+xFW-=e=x$!8Rp7n|tD)M*=my1hqq3kGCKG`PTq!+ik~} z&t~aW{In@szU<+iT^ldoPAQ+u5wB>rC|kbqJY8(p8GW}@!5*gTcdXg8DcoxVv1LuQ zZTy#Y77J-lSR#Qe3bo8_y9~r#ZYS9sKg!zs)p^>Djcg|n&*rLD1Q=#5pY`(`I9w@s z4=NIZlL$$PXt@Q|S{s&r3!3djdoOl45!^c;oc@O@}1lKuH*z@#6{EX-Nj)`t1PH=oZHb=wUYqM0&A9h&VyUbPD&rWm}DX@0A z=I}SN<;z@yz70Bl^W}01@tdwd;y0gJHM;{@HP@~#8Vltk=eP{~xOgpFytV=@+7rLn zKX%W6T(rhd; zY%F!gfB(w26D17dLbSsqS|qEJsO-4fGq4lf5%5v$?cwxSskIgi5 zAsVB;PR*mdbEgr?A{Uk&HPNDUUGi~M4jDqOMr%Wwl>sx?(IMhgJBj6@)=yCs*Lb5W5{98wae*2@=+L48iwhea zT5#bu;GzS?W$AHYf86Gmx$XT48GC*cOp|X#Z*=rpxh$??ocq#Wl)np82nzxVDPfp< zSe}a)mvWtrcy^xEjP)TJUYX{yoO-L}~J?zYUX$rrAj=j}bpbKK`d z%ReGi$nAj7#(Tdk3Sr+WI_|RP9|yPN-fBAT>g30An(dPoGOM@FmYDG00Ge=U+Ulj2 z#baZ6N6gXEz@`SHJ;Zf)sWV?Gtev>oYxhN0{;|h>^~&Lu-Wc5?{+-h%pe$N)Z^Y#; zaaAysN&hGV#ocY}Ap@6(0j4`F7#1BU$V=Kc1|S#lGbtG#L-p7tbG*LIGYeQ%3bA>n zYn2n58n6oc^%9ca4PM}VxtG5{@?gF1`+dmZZvi=%6*GEq3I{VxEBvZ`2DilYsp2n?kb5nry7=nGDS5-kV7 zeCxdpi)|UGVPgak&1pNqx{jturr;MZW*MY`PSa5j%+#pv#oh>=YMc=Y9Sf+^F`l5~ zI{@63e_S25^GAe=f-bzyCY97t9tnwq5Q-ofRJOyO5M&U*j7<`|i_8;(lmr-EkWIR= z!g-<&kOxklJS`FjJ!h!1`4FT(F`A++OR?;X5Pd)^CX*n zU?T0dS1ga06yq#nyLD?Qllg*5az8XAj9NzkgY4(cP#Q1Pq6##p0_?r0f^A0X?gNWYd@xh{j(V>oRX*{v_KOO zFY!7a_Qk9D^gv3f7#zg!2L3k%e{Lgab1XxYJbky4HU4&?Fh>cXOuWkJ44@At@%$4w zS)xk2_J(|!>F1x%?N>kA=6A*D%qLd8cCy*}h+O(JY$~+gy%N%R0j#D3ay};xDk*hf z8ImJeI;YcbDX37W;@|<(ii(a6_4!s|L=+hhu4)=$YSHYRipGY+|7TLT`?e^HqS==3 z7#O?J*xf|EVj^U`jE7TJU`*gV3YA2FXX-Ck6o}7FTa?fJPtv-)4FaXjTR0-=1~n)4 z+NVvz0X^b{bm^E+N8B3@f!H-}1@9t?eKZbywA46oTstPj;IZPCYFpj=x(2*xGXy#A zU^e+36;GS%`{A{=CApT{%*O(stA8~YlX9toU$)NEKHhdBkoK3=j?$wrR(;C0)7~Dq zJ6*J?9EcTuperq_iNw`i^V%WiXLpL`oTc-+zrR|u<8kP?Z^%xL#Ls?#fX=uw$IM5` z5eY($$MZ__+D2&Q1X?h}>tgMjO8KbseS#y&%ROpA~39pPT{EvQ;y-t3EQtaYtUBKB!u zs_wx8|M`|DC>8)Y;`Rd0;LBQnr!*{c@TlO`vK<&Pq+=(?XY^LfsTbESso8-pr<*g0 z_benR9rS7t#rKYA5NTN1p3Y+$O#m{ZVK6YrP`SL#TTUx{f01bR$#-Uxj21Dkw6fj% zt(=nZy{q|W%Zyv&uk2!SuGfIgO@<3BNa5$?z-RBxcX`0aVD<|&2f8Pv$yniva=d@w zGS2CF=sERDMZ-{#_V9%pfM)K{UrZL9-wvxP2~q=<*|=R71VZG+NA+|1W@2wrmc==A z9NI4bR`-$Au9CmXaqL0@^DFX55d_?9;XxW75Jc^c+pb<~#6?jC8#Ak11J}$#`86Wu zu-o~Va0ijcvJ)36bH`nc&yYQ_JsW3T}*h#Q*DY}<|bx8qy*n2Mwxh|3l;M$219MQku#C`xV zE2hnLC-`5lBlwJC4Zd;uWtBtrQ;eu*(kuMiKAC?Itk8@?I-5jA>zQmONn7ZcXwa*L zHXU-XzUzdVT#tvSskGdUIrB@NP8ss^w-ULHUwo)6%3T_PI5PvwClZO_TFw@Wn z7dC|&1fD0tYp25?B$Vsk4)a6xIR_JP2SR;XSx+}~JWP*Z_y^EGKKDb*UTH)RO_B<^ zX7b=4B0n}tG<>_GI)URlh3uZT7nZm^WAyfxwU5XoghS)6D!J>>yT-cD*|&W=y?6$; z7Ivo!(o9utDtGMtH?gs76^d}e`{w#CmwKNabL*BKr^oGDFhdwF`klN-=b zOTW^c`EdOBpv2U6@u4Lert33guH%6kxXLO8`w`*39k8fkcYW7+4bo!O!S31s)Pu(F z4PcVwDJ5aSvnXZ9;5TU1ABfFg+324jaXW3i*P+og$Ee1>lpsDsKP9a7)+5Kcg!#$_ z&UH%{1cd86CU~W3)#RpLCU^lc=4>7rQ>114n6V$!~H?UFT)n%(r&`j12jj8Xo8JjZ(&LiO_me39(g$}PbI$30(?6s=i%pNw^!C5MoJ&2*{xK%%+dHhvgDX~@Tevn0n}&M}>{lJK z{qovTKKI`{DkHp7TE9?42trIqaZ9BB3&L-cG#!Zn9$Is{4zK8s(J=6<*ORSaXI=^Z zbKwt=^-}Rw7FG2S*~?>Ef(`sb{Kz&D?p05d1pooPx5xtTPJFZ7{vH~;^*51LsR8cY zWZ1h`j&Yy{cx6`Y`&*=ZzC>{Gx0o8a+Zkq|>fv^Ea-4PB#ufJlTQ$^K0pm(QldRzC z4`$_pZnshm2DL1*sn`4=V=TkF=iN2~I$2~s{1~ydmR1m6FcJV+ zTC=SZ6{7=}_H+F>74>JM+TgtJv-vpkEdCKvB+l-Yj$#SU!&B=cSn${*!oj(P{%7KVSY5ibk(^KS1lKNA@);v3QQQ@XC zu`uLiAPW0E85S|1#kor9y_+B(HbR;F`T|9Glo4q=TNJMbdjf@RDw+!0aH)cg&B; z3xVfN++_s7QEu7D`=XHOX8%q9E9Cs*IS^-emIkVVcZ6@oEk($9)#R2RRzD`Juz1

    z((FQool+|PToo4N;y4dN*ZL3^l%&YB4iBLXZdm>C(5!EsZ!95%} z;7Qh733!AU+}xdy9Su*R6Z6dv0dk2)9L|#9n-0Y7;ST@KgGS<$Blh&~ zzQYM*R$SU;ao?b+#C^!co=FoW<4>7Hngww`WuXc>w0KO~nQx3v-U2-`Zu8x^T?Uh4 zbX{|*e^{3^reY50Z8utw^FcpZyWdzz5a8)LiY_;x!@UKI_u8r-DrA1|9#&M)QNvW* z^ynQ=!4;#u2Zs%_+`A42LoGEV>^~xGMtP_)&-))6FyYb-&8eOxgarDPFP>H%zTjOu z%=YEtcx^4x4KQD;aiyKac;3@9TJ;%;ID)Fl+(pBKc;NFDEG|V)f=|Mz=RquZfDz&I z1|1yOqyLSE(P}(Tv>-_X3&0L3A>#15_Dx%dF7v{!0IYH66$TT{1i1k?^wV}^VL|^e zp6v^m2@NG7rL!ukLknbb>@a>bBA#&)$KZQWaypYz7^q&}pjMdig|O| zM|!lH1M+0ubOx4F_1Jxz4&GcIv8zlunAi8(u_c(u%PB-zz!iYUi`3g7eGP3ZY8`jO zQ0LG`j$q~B*n1fjVP*AVykaS7?j8eO4RRbS-@oxvhqfk&Eua}z@z-Ak%sVIoiW$Bz zHPf}y{38n&LySK7;m5?7^K7|?ngw40=DIZ{aM0}Tt~s9X4?k!eFpS^iKXJeq82CQ^ z<5pFD-&2Xs_Iyy;pDI%79*VneXOF9%Gr%SojI>+xyUNCWAp2C(9dSIT>k=*KNAi^r zz6-Ea`}q#arnYg;Wr4*PUu0p&sYNCy=rZDZ8{9xH9-4%dLm#ZI1p6pD-HX}-Fol@c za@^jc>$R2auB1c-i%Q?fhPpsL!8-%aBZi`v-KNhP zT8#<@5|ZvY&yQ0p?uNuga<-I#T6;$ZRg~S4*{i=;`LWm)oz9w=46be?&bfoT6|?yk z>UzB@Nc>qT-8Xt7Qgk%$21@|@*37te&1E0YxN0=`^1NtHS(5GrVs5fxvBsl}pRqf` zWMnfedYUL|{&``h{h0zcyhD;hqx}k-g2jiMa?UL1(ua(5aI=@=QsmMGnv(&G&z?ka z{^S5DmjZ6t%fT#OsGkz30}KZviX6)+dR}@3%`vCm=ipbU%#; zzxA)YBlL+9aM4g7+v!jo?^8yCI@FkpEEpAdGUhg+p3XZqQ*?qv5vZvxW|?7gO<22| zW{oAC0z}_y1n{4-1)hTU9X0N?}+V1BULe?u0XyI(Z3ps6X)c2eTaL6nQsLe_#MUd)U*0MFF#CRI8;INs<9O zD6JTju}$zGNk>;04?jr*aL8p8`s9A4ihBiET^z-1^3S@k4j83CA#FYJlkl^*>S5R` z8Qxh2NIFUF?Rx)w&Vm6O0vWbv=|4LUSdQc&W23LypG3M5^MS|)jb4kZWgvyoYnePD zz<+zton8Ix@0f-|$DKz=(SQSBk03;?Iau8k(ZOr;AVj zEYxP8$K=+~xJ(`iJQl84BadCtv?K z7SK`qYg+mM5f*f(sv#=4paq~0g~(6@4g-!~i<&$7SHHvJ6o5}~O}j+K@QTPd_AX%EyWmvKiBC0un_9fJ zEbnFC)9eZPry-kz#+o*}G~s`fDl)z1In9MuO+{?J+HF%PK*_8!#CN=f#pySav?E#e zrOtq^elP|+l&kdr(XiaM1G)+V&cVwr_U#yQpUFuCgD}NV3V$@PKMLg1>OahxshJCh z_1RKxowpizuCiRP9$%`0xitj#P9cP7D6KDf(S)r%X}yCD{8kD5r%!(25(Q-uKD-4b zUW$b%aSL2-_-YLc6PP== zO1L=2_lh7i*Yq2v7eJ>{@lGZ_((r=dH`;%;*WYhx{9cbX$me%i$1SrY&N9-{Cd22~ zUpq_eTd5Y1@X$s@OfpHzJuG`O!X~iDJYI2V*)bT9>vX`kyIs)kBd5KTS2h)P>@U z^4lPQ*3KaT9pmjxbI@_}zS3kz>i2H(+s#m|w;fucsUDEz!uI(Lv0YSUw&Al(CfZpa@ex*3G-HT*8d0;W%t*Agq%w$^WoF!pwi$2!{HJ_;h7Pp!pn&F6J= zv=SQM*Uy~S@_Y*gqg-Z#8GZ_bX4=0pN!4T28FU^oAK?N2G=7?WzrVG)lZHq@wqh`N zACMa=XD+Ym1liUfN|b)y!UY6u0YfcS0y?wrQQ%{<$5C-Tb80S2&#IuQLv@O_bKFEHC zRbO_l&!jT-UqeFIByBfJSuM1+WrJ{_1B~c)l~b1FE|1X2PWI@VUxW>5%~k0Z*HyX7 z3gi>zefgqbWEwmmxWsX0d)q+2Mqrv{U`iC{-9QYBoYOD{QAbvVWxPqpcuGK z1ilOX@C#qWDAw?Vwlt<(R!~D+*)PU)zT!o5)SS%q36}J836zAnE#jNOXbq4;!x(FF z2>9F;)KP3S%g|~)srj}e8U*a!dMC-}GZE-9dC7mgPCj;d+eUToBdNa;imy(3fJ=gq zKsPcticV$KK`+kc@$I1VG<15O7-wbbiIwUDCfV{FL~%GU(yN- zyWS|Q{@{OrPjZB}s{yP^^3Co_afidTL&{q0Ht*q;0;Hh6=aJnC=;|dV(tNj@; zOWTecrZ}VFLeLguHgH}JnZS+lHrQNm=?&?#nj?hs;_k7-(hWl~RYTZ~N`UGjF~ll6 zbXZgSL=_&&+b0}UhV-}1&i)Pg1%$LnHdgCAKobiH72ac;_(B!4a=OSWn$y;_g-dGG zf1w^Sbl8TG^}2q#EKENrq7j*eKU?K0dcq?d2cJu6+FSA5IlVU6pD@HO{5)4!D%t-V z?d8i{wDiIDKEpoFJ6c~%_57e(f_6x;DJ_Aay@D{2+C6p zP25!jFQDL5$T>zh?_@qk0`q^kG39xhU;1hO*Z*(x|Noa==BcV_%)jg!rVh>10+o4( zXncLyZk=58RO%gru7@_$+zpF8nvQ4qca0_e%?=!09LQYQ84fNDWPndp1Y&n6a8`9A zP5qjmBcy{ck2F5N?13gHXeWpe*tsN-LA60PW`=3u6f#3eSLkWm`d+3jo1XZg!j*lH zNBoM$+Kf+n02Io?Ao0K_SZ8LFW&X4I5C!133TqZ9dU2zQQ(MyQWBg9y27Ua2nO^c4 zlLL(jF|`0(=-$^(>yfo^7e-C&+MWMF=I){t5$FAV@yQeN#X+j8A&=2UNHBa4!A<76 zhk`f}4mg>*kqKjKT>}}GulLr=XBLCNlAXzrMKth^WEK3EpI-RwuX>z@_K+wb*0NZw z_K>i`G%bAd6z!)vp$*ht+*S|NM4?=)R4<_(rhVE`)>fIdiZ*QKeKdsV(+;%W9-_O9veH{+&JZ~4=bm>>Ucu*@z9M+GFYN5S!duyCs+t| z#cCm`Epc%(V08F?90~^y#nMzhalr=v!!9#1LQo(HxSBg2{3C}KzVTwno$17^(^^+} zPwod&@9zkYvpYQPb`;G179c%UMGY8&0i&Rg19-~85NN%$ZN9ahT4{|BfTM}OPnpKj zp(6EmE^O6dVF^!{>-b68^qLfBh~NDb_aNFB4u+%{ItvO8yNDZCab_tQc@jmjT*TTF z@XQk7v~%%79C_5>gf3!-J_{6*Us5Y8Ea|S{SdjWaAJ;I`{+519`2Jy)nHi!CEMnZ| zYU0wUWqWr*>jn+@<^o$(q+3=-@hS$6K=~9*-K=1_8gA-Fd?>+fcvzlAQ)? z?q$0c9gZ#5V!h2yW>QhEzxKbMo9Ot0-gT}=iysyFk2vr-EaNKIw3+f;7Nl{ZUk53dQ-zAfC1qfUqu!bPivEm5_1GWFPH5;FQkP;o;!dtcS)N_O1w z*scr2!BaZD56Wq6%InqT&ih@qAFnqtpb?|9te!#zo#h5z8xUXw`d_Z#q}w!X^D++L zzEj!t?<#C$D_%!GJMin)vP%ETeaI8=2wrkfNAx}EMA!mCJ;{LcuU`!19TgDXFFS0z zT7~{mLd_p&w0m!@qob_>c-{|04>jF|l>$&ng><>rd|d(&y#Wu!7w5b)8|5O!uB~Btj zU;gW-j72-hm?$4kxmw!M!(8mgh?#0ch>x{ImOs{o7{_)@R2|tiOUIjeTKINnw zudd8?y3J}yH2?`1g`@oJb1Be6x`bm9K#l&bb+HnG#8c!SurQh@|}SVN<~ol9D@6}IKCm-8Q6gTHL@x$RTU>HFAaN&T&8`Ii7h zOMeTwbiTyv&krLZxb@ZM5{^A;wuzl^_<1(SVD=l|qO2d$FAo|g|GSfB5(8vw3|!4B z74&2<%gh*Y%ll){fQszU(F>Z6dyJAaAh<=rG>TbtK*TuL?>nHGo%c(MZiQl4VSd9o zLjJW>meozlwm)R-Z9pSV&@_R~pPBZz2dFXM&#wVV^@kGLTLy)^6MDGbOpNK`Zq0j; z0@7#T-C#GfBx`8f)8%(5evbp;{^o3DS0EQ<2ypplDfgq&&7J^S%9{_Y|~^6uRm6IGTMX`j=Zw6>WC zNzGZbBm2TL&0^nBYNT1@c*$O+Y8pp;#uPw#N)8gfE5Azx0wkZ)-)VzeADn;$iqt#u zc6BEpE~H}=if>N0rKh#G^P{vvn+48)#nJa-Lh9udZ|!50p>nFLd>TuBQYergQNd)@ z&o&9Ln6fvhs_XQ|%T9s1<-6B#mw$Sv;h$sufXCG_G_HA(#5XtReWU?VI2c_IK=^g2 ztK8INP$2Z~6mWw}$p&Q_P>zQLFM`1x50{l61SApd4mx*a3Aj4+kGwT?_7+X5xN0r@ z!tdOWz>D?Ap1)h=?}x;b*M?DGmI?{in+HLIr0mR?a$Fn01w!AFu13s+k`w@s6+Xp< z@M`ie_Xf*g>3yW?`znUT2Ii)}34yUBK&}IIz~kMH@QZuS$!J+S>@(Km*7HRHlh%|% zc5HQ7+h|aRxtf`g)LuYZ1oO*U>*~WM?FU2?eK{bT8Uh@@ltIfCx4ClOACguaf_;09bH4l>AsrH8&26Qe+$M(2-fx z>6;X+l@RY7wgqpcfF;-oi<4r2y~=mA>)S20qOQ2)(9u?zYu>zj|BhRAHDOSWNYbm` zz+=_RqLXor3DO8n^4a(9@V_7Y*LhrJ)NM@Zia=2nK|p&LnvSU*iP6DYYg%ELVwSv7 zY-%E#L*eTM_|augt?xW}i#iVMSDNda92(RkK+a960*LvSwsYN!vXz2dwMv7q-Vfe~ z)qS#N-_4SPL6NEBhz=D^+h;wRE4eNInr+@@?{+samGB*g4jgyFZ`6%+=5t(S^F*Cl zcWfjnGoa)rFVe8sQj@DeNC_QUonRaK%WmoFmw?!@4S7ify3z3F^a|s>VkTF46>!M$ z)ODSDyMXA+TE;hfHFQpk)=(H=OwRYJm*1Y%V$hYMzq`rIG#p8k>bzBW$|_5D@YE+; zczrIPpk#u1lUTt9TSo+yy)2i}w<`udNA+^+90?$8*}5^j@OD1OPMtz+U21Y1c0LZ0 z;nowr<6lFEs@m7UZfRZBRz$qt&U8RFM`|?-!_Nnj0UYoXfg3U(hFoMpjIJWg#+-cm zFjwb|fM?Zx*s+RU{g?!ru;LOC=dk%XUReCxEY<1c2%-50@ByGjnD4iZ8WS8VKzxpW zesPIzn595AUNU3WIvV(0etcWGdriB(|1Nm8MX&Ez%r10IjR4%l7ZEmFiexZTA?p zIIWKJMq#*yVTJ}(x4&BGsf!gqHY>tbNz;xI{iqk85z;7W|9J(-^Ph{BpOUJHbF~B|I)`CEHX2SPyHml=sV?M$$Xl(vt-cP(*>Wh~SwF(%8kkvRi zt@GO-YzVKh11CE>yQ7N<`=%O6z@1ZPpjdeK@C7D2gJiSo-bQ9cZoq>UGy%ouSG<=v z6*TG0(8b1tW$11b=DUWzSFe7zJ|B<>mh7?Sr!kxY;|wGme6eCV6r@yM^!sxnfG#hV zv+nq28c5=^?KlYNPhABP{I=}0t2?>Sg+EXE2|N7?)_P!)G_8k3t*-(=#PrICHOP7a zKrK!(!)$L_q*CuU#~!Q3iai_zzN=i7v38Xj=8kh@;rlM#k2)5=8iYkvmeRF&HN|E& zh|@{p*1VSGLkf9g#Ef^yeIO`5^v0=Wu0-nt6QCMf7#rq|je8ond?s>$Q|CUW7W_r< zGZWxk=SGIy18D`X*-c(CTfv*1o((ZKo(A2X0_mBH0!Cxp(-J*l&h9If9S`)9w)Lrh z5PZE1)3K9fEWyoqOE) zJa4Rb@3p?;*!#!#)9GZ6x_VsaIeI@cX0n#uWN7_^nedV=1gElSZ1)T=|c_v2QV z1Lu^yHLDz9KbJYJ=+1)zwj+Gaa7k#?huBdXm~2Vj@+t;(57bt)x2?9@xl2>cy7r>sm5TjluihAC=Ydc?Oz9m&#FjjZDc@DMghu{pz!wI%nb&vpP>yZ!}0O zg!xF@vW!x8mJxp&qqGU-+v~ztw!sg_9kM-b8#AWTjz>su9IV&&usAm8b+LC}I1Qon zfYsT!w+9z)cg>YfND&?mm*9QycBo%E_^xcc zDYvy`IuQ4Me1;X_>74$~Ga7itg$vlkV}Vz4WIHaZXSYnlG&Q>F1HGh z7oB>=-z{3i*w%8{8y03|25lyWuS6cOIM#2xTq>BGCGN8s!*X{8*jT^Gq|I>t5aoFU84yL_>t*QRB(rxn9bYGM=7# zXPo|r`XQ4Kz0bW;B{bTdCH2b9EV+E8W5);%&y-Q3M>Gg?r1{s!)SVUS)S8n@-D4go+Js?s)TgbXrnmp66-> zJ6VYv3yRmB%w(lv?3E%G20gJg)Q0o;oQT=;9k+Ngy*$AP58rBIDwkvJSG}xeyO?G5 zuIj%7=Cghs^#rsme~O_Uwf+Ih^q&e|{QRK3tp$HR@awyWy>vIV{cX3c)g9c|N9co{ z-p1<{!Fic$ROgYcOA-0^Rd+CDu&6SL&yDjE1*^3@sE!>_{I4Qwc@(AtEBws^d)8U3 zQ}@C?B)CgFRlpcFWtb%xP;4LOZ55HsS)GQB<~bMNF_gHmQy02y`Un*=V>snubDgBAMtHgs$7^UW*^CHVtQ5bW{wHe*#{hFEk38_Cu|Q z-(lJGcB&85f1f)vjQq+tDi-wk6QhipaZKkg5u4;t6RV zW1!@ORI(yljXJZfk-v3W+Xgp}J-VV~w!7|APWx5`$6V&Xj)DeV`egn(z6{R#zR`gz zCM*Y5M4dEi!sKG25mS&Yn0D_T0Da@Yz3d|20u45_fM>G zKVcia7nnNv%dFK4(|?x8sv@%N)=%{8P%2cFEUW;|&DzKvgV6*V@0p|0F;@zkV#k+$ zgn6avo)IE~v$nPakB<4iZDifaT)6%A zUeRzHiRso|sQ(Z?o^jf)T2ltMAUL>hwfBuWF^W3t-A7dA65jGueL;VTRuo_m7?Lzx zCQb&BpvcMBZ@iVLkmCYyMC(h$!jdvleY_a9c%qbEFY~j`Rh&CcuU71(?tAOWMw+zx z_uXTrQTLHrYkcPOjG})*5WB~vm&dVJ?cq5n+UTEUyizj#vEP-FFHQ_ zKA^%OZJ7ptnheBHbA!eD-2i6$Fipct>^QQaR@YvJ`Y1G>vhX;2&M`hb&sQ6!F(Bv6 z>q2{`U>C|JYJ@w%@gw0n)hsW(|Xpg+&x4=t4J??pM}o~b53ev zeb6?kNoZ2dR@iC=mX6}nRI20eUOdNOISqA^t*E@x=Ie%qF><|n6w<6(raX$HhlyUE zVLfR6Ip++xChz$Q%)_33Zfz`Q^gh|(NJZ}Vak3%FRKXR~WnA* z6vkihkWo;wtW%d?Z+E7_$#TIY@gzae6fX9qlIk4RC49VH$lX2g*vxlQz>#93 z8k>$Q%O%Jcs(Xw$b}lGb#C${&AXar#9$xGO=gu>2Q-N@0}7fX!*3%vnFLdB}gGf(7}ROK(>gzM{liP=d;< z-xE^RillJ4jo;+plF^7$@0Jj;$+6~1Fip^{ta37{7d_3Vg)f&QhM%OGGXYW>`6ny@-#XNN^#IE17eYu#fogdsvvj#=scz?MHUsJj>vu{PQsU5YqMjdsU+;HP?b4 zkX^@ZHuF@I4#oU6bK7{Pw3RZyGL*i@@vXSW<;KXQ4*oAJXhzQ#aZzIlethh=7wf8OqedVm|DMHeI^A?ewd3y#C!X(2 zS?^B5bSLIkg5XlclhxB4=}ha9%e}VMMz@ygvw~r=IQG+fI7oEk4OR3rc_;VI#Pwdx zALVpnA>dxmq!1@Z=3}KE`_c~F3iQc9^E(L;JgL``hsmUqMxU?t^Gcs3;uzC22b$~^ zD&kRp@?+FXl`KHFVZur8`zgpeI8v9l<3|CXX@l7Z)u;vVzdCaz?lH`0oV``6>p+2w z=!H^cu{eWGV6NTcvXiF88htD{!1wK!M)ovCTZ84eg^O>*&-<;=fRSnTf%&aym~G8G z#U>#%JTRv9g62yoT8s}ptz)sil*fz|q<_U0PC%>AoM$h{hpSzcQb@4>V4 z!h`iG_EH21K3UXEVoc+dl=1a}Pe81y>#Ur_4$0R7k~~w1+>XLn0+;gA#vAzVREaG5 z|0S8TX=7Gb*w!pH8NK2-k8N*XxoSNpcN*YI#Kw^ZjWF|uEBg}gX!Y3pty zsCxGX8@60le18ELVRNb%WOvN58ZRlZh!n?giO!h4IS6v5RL74@R)RNG%4;~*zWTqo z4D>2D&AB&=fl)UF8DFbq%O5|NxUC4z3Dp_o(Xx|ic3fT-mfOZn+SfTx+ivDWmWMNVnCZVQ|Hdz_D#i8uuf)t`bOhK|B4f`Nqp=cdxp+RAQ%Mo1tz;*P0+Ts3q9a+TORp>K%FzSDHw+!y_ znLeprzBWX$UDi)*t8w$5qeDMO-)<)s|L5HV8}&R-`SLNI1e3M%ZM}s|>@=a5ff}y+I9Szj(jf7tyCPbdwwZQG=U) zEa+;K$tkwSr@@v?f20U=5n?G@+g!y_lxw5#&qL3f$a&O{IE~eo^5whYIA@x4Mq@c{ zBApTE&eiubSc^k4MTxN%9{T-PyNtiG2Wfo_LLR5j^+u=*xN!zwK5P1#k@C*IV=+JW zMw69;RGLN)1l`_OH;SoGEX$U`q_PInadPJx^03lc zHYe1(@L0|qiR)~(_E@9R`H6Wo8LD`hH<8~gjK(j>&EMV~P}tUu7m?((36+MbcNxaf zlU!w9oeMG%59<^f~A&gE@?C^Fa6R^}@{Pt&f8O z$Z`i5ZI(&;t{<){#v5KJ>cVPpQ?FI#jXej&t}hUmLD%;;mTqm zNvWw86ik}da%t}^=j?kaU zl{@uvd=$j;URu@dyzQp1nzetIb@ya!+eaNwQa=s*+t}v2BZNIYVyBaKbo_|lU?$V> z_Lem|iXXjRxcRkrtuCE5Wz!ukR1CWaB^|v3#@8`A00gySOOqbk(TB$@NnrcRbvV&@p@ znZI-fr)v9x%|&hM%85YfDXpZ;<+QE&iMd2pCv@HHKkcwqcA_pzM%p9v{MTr{43a zRmyG98H$ze>a9**LqvO(j~au#&GDKzR6p zy@y>#Yrk0LvnVfW^)u}r6@#Lpt?nH@czZup9aa;7YAMKUTRh|R9Xh&wu6*8*8>rG0 z-_A-EIk?Lc1wR9Y$%KyKcB406%HMkx=jk6W6CdjUCo9P$u%U@`Z+N}ZHYw(7s!P(h zJT_qYSZzw}o-Jst)&Ts;WziBEHr;{K&JoVXU+;8;AOuJvGAh3q6AtSrW~$O^QdyQY zL^kxxtr^Or*6#=6eMQCr%~H@5|G7bT9;n4Y#E2!HF&f(E}NGcdIKH* zey($^H|E%rA>kDRx|DN;Hg=J`zk{BPZ?M1m6LefRHj8znw(6v%f|NotPEf_paFCQ+ z+Zo^bW!_Jp7I-FW_NLRkJvk`Y=i~i*1vDKl6f+V{f(yP#=)~2Mpk+>23pV<4@#o2M zv_0)RlT6lDZG;J=U+==%D?Ot>q9SL-bnBA4cn7FA69M=Wm7?tl*Ke61RZQ#|5~<#~cNr6j>dyl$Q#sfCpZb28Xnz!>iiLEJ#;nMS z+wO-Uajgz>NiwQG-?U6V3roq@k^Arn`LXh})U3nthi?``VM7ONek`Nd3-{2nM}G(n zN-~{jaQDt1IFO8-{Vl}t#ungi+QVa`?ZT>po&|?O8`Kg%tf8Nu$rpVnQ{?}ax7nT%U)Tliwx~o zJ7(*RAQncNjy4Op_qwr4(V67I1rU-Q&>Nm7ES}zdl?ylh7X5@wuNaPwAbzeXz}h*O z?aidv%4WV)QX+m5ihp<~e`M3!gH$weppirCau|PwbX$u+;qdr-$q`E^tAX%yeK#DF z!?STm*oBE}s~};je<0HUayL(M(}Ig?kJ{S&CNe=6?ubvXOT^{EjJA(ETyl24rSU-D zy?!Yj9qfi~XJ`H~J-w!ghkpDO>TtI0-9CiDHjVG7ZB7Y+Ds|C8Y{;Bfx5*}m+&Cr# zBn#@UG^VvMHo|l11N}AP=TR@r7H0%Ywu8K1TqJlyOba_NhR@gY>wAccc)LRH1bn7; zRFnCpkol624VPfOQ(WFh_&Gd)TrhKEw-ZOKdd{EpU0k|IWfNoAJh%qVbGQgp;r9-W z*ohboz!v<}Ji96+2v5!~Tz5W>$0W_Y3kMt#>C;(_xJGYYpRbudmx>C=Y+?su<5(X; zxJJqNP&rm|4cy5nZaKT=^85551W1oJL{>K1kjZ{`StGVQUE}sRZkcyG-a`wG^v14w z;_}S)teEHNA*E;xic<7WO-e9aN2%pEd!T=&?oJ;U3vI-SwPo7=E0%g4;+0A7 zh74z~uP@%NjdC=>Clle~?<=ASn>OafQtf+WRY+I6EJICzFG9U+J99 z00^ROCuZg8XtjM!^`;p~$uFD>_WM4@k_9`NQOdK7WS6)ZBr9J{9gp#yf>b8dBL^FM z>9GZeJ>l1SM9xp~%nOUC{fP1m9O1fL)B`#}D!I1Kp6og!^RplQ!~XIJ+0JLp|ViflEnF-%xncP0BM_94@d{OFy_m9<(o%<)j@dgl3$ z>dr`f)6v_!(q(=&$d$i*O;5+9C>#P@29R2x={8#k`wBNVoQ_v9o?P`lFGQW<;rx#pAi)@JZle_kiw2|1gY} z1GEKMa(-4zmgnrtxh$d!^!MSl)sFWI6>@Me{MukUgk|odCOF| zIdDH0l5ipBBQ#!|2b2wEpKCLplDy-apLH8rA#;&sGBC9^RH-8!`lKi8U8{H5eVKdt zT9H-v8pJyHRGo1~94$OIcqVvsZY#m8?8J0~m?C~Qd6~IM1|7u}lPEWExIi#2CCzpG zB_@W+g7ABRnpSD>M7#Vao0Qq`z;EpDMAASCx-Gwxs`=_82Lc$etoIB5TM3di(Z%O+ z-{z<&MAGjT1Vrz>gJnNO$=_eZUt*1D9M1#! zpHX%dB`=!+4ME7AHs6o=D-xV-J!1>`X->~%Z(}obAmBW;%Er#Gw3-0G>vS)S(3Ld- z^@l<<0!(P0c^(;_3xplFw92I#Ox~CE_{F_cL zxy$*KSukYh^FSz$Fr1+e6O}{!RT~~^C~+!w=3zgX4^ z^lDEu?k9GU;Om*yxs8Ng-MrirS!H|g@z5w=&K(+p{uo~Jfw=%I8KDq~YD#S1^QRqU z0N(D1vxJANY~!{3b+60YbOWT1yxU|QCAIn?N|I^K&{a8`nA|+lwumNuK_{Lx=GP`v z$)K793CV)#uK{Ezfn`pp>Y(6FE|6yySb>Y}o%WPINAm8DXON0W7SeHr|01~T;#dfx zL4LTc?6NiSjZ(+|^0uSZZ*Wy(_!{GqCO%p)zUn%ku*wkZ!NlX(LaD{bviD81bu#;U&E0>$1# z(jKmRiN={;)o~|mM&uBboS@>%>#I9Wij5kf9bV*iuUiDYi%-53RRnpSloo~yy_NLz zb-dLfkxAx{-y^lQ+fimD%hGf2=@BN82CQ_1kA|EgU<6?3Sz+mYaE^WB{BP#QiX;?L zOD+BUb}hKkUlZk?3J4Hm+cx!JEk4dLyhb{IlXLOy%F;91q8ZT{Xh#>&jwvH0#@dpA zw#b0@dSoHzU|Q+s+102_Wpbr|Wvrv<*$Dz&mtOHk_fM1PrdRK)@B-Eoa~nsF*89|S zgyy6{k@O)asGTMA``R3+%Xvt`wkJ)Q@KQaZ5qS2q(vJ81n9dW2q>pDcLL^FEs~xF4 z2kU!a0S{)=2nH977%|m(Sp$m|F&(jjxwoh>B831kNh?c9g(S1+*74HMVPKXsg&d?s z^=U>#aPcx2XYs%5rxx~;ar&ye?if)z8XX1C{q~l8)V1#Cjr}fdlPbVbEX3=3r4B!pu45m8{pku3hv+1Cr8i74fI_0pr}dR0Q(3NInj3oxNK$E{?zTBjK;> zXZ(1E5hbd%^+7Lb)dz)un5hzvsd7#lf)(OADEZvkoj9jBx{X>_&;C%_pK844W#^Md z-EcU1cK>1!=PSy-_2hcWKwJHJ=~l6Kc+32o=71wi?cI)D9Qd>(HeU zGX@!lq{rUaC$BW*XTEu}&sD08Wdha(n;HU{@)SkkVuEwrwc=YTrG3BHI-#~9@t3uxtZJZ&L-_YZ4bo6?Jt?x+D~{f} zpVTY49-=^Q}L5>otLKV%K<9NAILVv?G5z#!<-sax76Gr*dOzr1)v z69p|BwhXTCt5*=x^7u45u>stg$}_+uFDQ|IIEW24WH@YXH?@zsk=q!Aagj~Fzgf0bkFs@0mk?VKU>cBItu;*6@I`=wt4XrF!BsJ4C#ufxe& zFYF@cIGd4|Tcd4j^_=SkgdDsDv2d!oD;i$`ubQluhczIWEbjsrCa4`Y=30g9d`>Mf zb*?9a_pQR}XM??{`O~W%?i%M2hOQ5V;TjI3i@GhQn)R97t4~Hq{Zhw;t&#t2tCU_g z5YMToDEX^Lu0ur3ovM{!%WYVlvoe#CmSTuW(XGZ6HmJi-L1^Tl$L6KC*ku(RB0S$gRkBqN+zy z1DKRts4~)Ck5uX+jPA+4f%Ak$Ug(iWn;Ws_$VJzh`JRy-oEI|8tA6);mCA;SY{_-L z{(8Mh+N(PIqDE(0PigUuYK+bu5Uc)@m^MNd<_Zwx()A2=ceJ45ITwDd-3_M?eeFGv z$Ze9SGUdu;M`mGA9Y$glifh>oYb9JpYt&Xczmu;o$3HTFN$~qcF)Op9t-i=qpVgU+ zEGE&O1Xb_!v3Ns@S(-_p6qlqlNpzN2go?2(vJKN!x1-haou@_!x=G8zWW5|q5?+DL zd>jW)HUqlk)ndIT`9i+a(SEqpbP%78n%B#_N_StUH|O>vrcu}<3z3@kbIib1>4sTn z4=qRK4_&pDExK9NzCgzk8<1kNReL(&eOx0VWjJZAd~vZ^ll!}LaqLW6iHbVn!A9U1 zO~jV~J-oy6w_Y`z*q_{H=H=kUvaN6bMpZZ;dIk4?*W!rTJbQE8@s+oUK|Wpkh;>z| zIGG+RcEqD;&Dq=oUZDvfwMj3&$mB6<20T zP2T$}&9;yKV4p8vb9xN8b8M|W$R4amu3%g!p8+l1TPDYgugS140mOk$ z+1J)dKn?s_d=QRP(MREny-t4(l!<43$wC+QOnhGnMx}SfN79amV9(nV2{Hj-+7#uCzCNH^)Kt;D#ea8y_70 ztG!+C`c*E=0RQCm{>i$|WT^GW^QSAGO$Rm0)ZD$Ao<-VXO>&|o=!hiT8_kj~& z|G9WfL~^RScmvTW;DG)fYL55{E4Am64HwxBqF^IBkbxNWzQ}+re(4_jFJP(v3OyR? zvvmTn3j;uS?Q_xWGqys+(xS<}^)zcO1saf>4nl76{mUp0$jTsY^gQae0 z4yB>-Eo1(h@&7=dfU1nuMDG6qR#(#iFvAQ(yGy;etc&kgrW+7dAUFcJ`0BsTM0{>~ z6#!nu&R)f`3e00_CmEKU{4$vMz%4Zh?d%ZUjeh)pK%GIzB$sv#e8V}6##G_`Kj36% z$QmfV`Ij?>#MSX%4^GoA4J`Xu)GJ=kN%-Fn?nd4y{{Jj*QCBvx2b}QSM{fgYJ zBKBtrac!qRGdZB{S|ft*O8gnDH6u?dC(YC3^B-p*UaqY7fZoD;pj);9>Itvh`M9~t zZ23qr9RInQ6nV|sXQq41XE`?{xm%EZu~33gc{zYYH=NfGxsvw30P{lr0_N4=t4K9L z+!AemNVdK~k`mrmrxvH6q`m@*9d8QN^7{bk1TE-BP};^dhj)NwGgEmT#IS+{8R9xM zgVW|v`#p!&=yYRL{<%hASH7k~GA9R~MIf?9L^GB9hiTc`AbDoRiD(rHp%hriJ>tYX z4xTpvCZXF)Yk&)jLB#Xuz+-V9$UZLGk6F0ffNas+Cnk+1B|?F1Ej2<$o;W8+rZGbQ z^Y;N`LCzJw{Mj*}7@5@2?}=ydX9qE$>wU6S?0k#o6z~_BLAaOhJGzeqo!5*m zH+;Z_PFY6G$F+CiI$U@5O(ump(m~&^8Q`1KEYDLOKP^)pW&KH$a_5T496WH zqe`iZkM|xtAEKVL%+DwhFB~s#+z}+oAeI3VR`uzQt_=-@ znR)b9SexoY#|8?S4K+kBrZB#D+a3Igu8qem8QjV9aDyHyB9s6^u*$_?vX6BjT6qP0 z$x-(c*O-VthrMWYu>RK$dnRv=iX~+hOQj)g=zaLNHWYvII|^?NWrbD0m`*Q%3m%l< zth|19r|;cBQB+O>Mx)hpwlkw-nx3u4p(*$bkKO>voe7e+;u_)bsTMC!7FUyb)zGk~ z-q$^3_Mbox3q)>OA(?k41kmW9RIxic;CDS5Am}bp%N@3U>rb^WkA3EU5G_* z{l^LDr3iEZyk!Y!QOVk;LsB@}F~5P*bC^8}QW;XXihR#xB!mW#ojQt=S)}S31H@%`KoysY(cf^ zL$@I1Yg>gwFi6k+>(*kXA2=H6=dFmvsP=S^ASa}9675LV)&n6<2ud zPBvOQ&bj+eMJKlF(R*`cnNL=UJ#S}2rUgUBI| zFPy174o9_Cz%^LyxTB#ySjOVrS6hecf~*NXo27MI!#*|$x7+|-(PzMzY#yWv%QXNv zgX7PN9L*!LPY=(Q=jpPLf7yldW*lD7<#-I+m(*LfzFv_zAANdQtE$!y%>9pl91!7y zA@6;nnA?AaK`qzo(j=TdNvwS(8v?+aYgk0!FgzB=a#U%%JU^~e9JENzzVNL8N3mtb z_i}Hck_j04dxH1`=iy%kmLrZ@(6a#wFH0o(v)3AE{CflN19Ry00^bavWt=BS!VW(| z%%X3-jGJQ-xGGW+E3+7E2jg>f$!K8#kq)*%8Ex>5Szy@~ z87_&lsLtihN0s?qo-``3L%hX#W9avB-@!>Ahl<7n=9iejdGXjIMQb#%IPg>x;p5o9 z4Y*be(l(I~t~UISYdr9+5{vJTx>Drqd$+gL^Q|w4B1xn8TuW*z<1jRD|7lTFN-A7Z zKZamHM*`LikU1+r>Il-GMyGr*kfRw<4D<=X1uK>fm3%Hha|RJs%iwt<$xm^V(qBA! z4JINW{}0KN!3MG?V=D1?G;l*W6juZTk$>*5&kk0MaMMuSUjQ@2pe%A2ukYc#mK~C~ zUF;6_`!l!+F^f~;i_y^3&SRy4Sip_kg~$`LpFI$q0oKyoB89JS_$1A~MOzj$peujq zwLJR2oFCx5W3?h)2K4j^nEO9I$KCGVq_(kZJ{+sus_akvRd9C!Pm$I57|u4!X$X9n zeP(Bk>>bI>mrH2w0C;QWcl*v8peMaw-d&b;uq#K}2Lux#!hnxOr4#zzZyL|M6C4$B zRQ&W6v|HuDWm650gh9q&Li2Y3^AExKD>3^O7sg$|0Yb)i!`x8^XzJWup_c$R8VAr8 z;K`lsdjP@VZ0Y(loTG*5c;2Vgd3{(WN?-|gj&`l~{!t7|dDAdEbMJg4YvAK_0aYOP zn2jQBiMn>o;5>yZO=p2yq5&>7|MD@-filwtj2;BkD;M?C;nPUBMoPMsWLmNUz68H=eI z)9IJJ0^z`_gZ0)>`VP4D$@9|!2hO;VygTu)m0`Yl(domBT)Q@<@gha z;|RJ{>%A~-y|ElR4Hd<~WAAW+Nlc8~0r+Xf@cJytd1FLA>cJ{;iskzhRyK0FKnEU0P zQF}|iGaXzOm}6p^9o&Cts`QXJo||#2WovcMUKR9m+2XToC?63sLIVP3{mOTY*fJg2Dj7mJ{W1y9h2E?#w z2Y@U#HS6>d#!dBR?GFJ7H98oGDSnOd9wcA6e|SN}3Bcj0GJR84NBl$YLHTMjHU4&SRfzYQAvD(+yqAd8I{)IxtXXS2$H$TYZMHmnMJShR<0jHPR z|6Lycf1gLslZZ&%+3fv)l@Q)AQ#e8X#Nh3pef~rA0*p=AzjC4ci6}kRPN_Zzg@Csc zN?15=3MhWEj6;3F0#=hTs_IVwB^^j>CZS*ZlOFwB+2t>~ySO5Pzq}8~Qy*r<+!T!1 z{v~0Dj<@SSYd%k4Uq04y-IQFY{rI53Wty%Yo9I&k_7Dj@RNncwIN+}W5U`Fj;1P^H z9L7(07$6z_muW*(45%%p@gdM7Z_rHALG9uNrMFK^3>BdPR2=dCpSc5GN|pzSn1u~D zwNAqu>G=%U>=dHtrL64!qA3Ceqbb-Z-EPn<3^S$YIdU)#6}zs@hai0vpPw1vfjMUo z?DT=YUTVM||G}ndzsKzO1Qo6NE+l|?H^e`*B*46Ub}(fsm4Hl=NA`ERW(0zY15y04N<0!wNJ2tJa)s^*(l z&cWjnK1p(v?$UEU*Ip@rxA$QEyByz)gVg$1sczOapyVF?DzH6OQ_R|OvQ!DW5YNuK zW$ya}I!pgozdkeaZGi_BroTSLdgaUiklOgBbo!1iryzH&ToX*X2Jgr3(m!>*|2pZv zWcm-yVs4Si&HW&E`We2!uTQkw+$RtvwWfjlN+q&{gB~ChDG_i7=|tk}?<Ag}>$c zyHiB5&S0!Kwz`?&=7C+v#euzk{^v}kz?qUnVdnAPSOkkdX9_XE2h&PfVnJuBHcEtn z_OacwHNoilIG! z6i$FUevBFt+Vuarz&{EXPZxetbL{=Gxv`5U52I50({&>adpJE;>gYeT!OsL2_5GWqt6RL8P zVrIo)8X?Ev4t)xfc&OrlG{S@%mKs=4QYkDh4YO9(`qK2Xarem61yF>H&d#6r?%BD` z7L_U29Hw36HCh8gRNli^AZqZKAI?*BncNe3|G;cC2?R=OUciWan(jqG)`M~Kr)h9; zlc{z@-P-SuqOCEN@@<$D{z{w@$z&|&dt%lbqf<$ z$-a@|B|0A${4t!Ly!e{_peJ~AHVZ1rv@BQKErJ&~);)@Lf29jJzPl22S{?W(6JR@C zdwXN&8+f?l=k_xsM0Vulv@b>fK2Y0?4W&iIfNXpPt@uUd@cxy~Zo3(Q z=AmKm`U@)v!kY$?xuifXw|``_la9%Id@ zF7bLu#%~u7o;}zwef~1vVp~7Pr?@-8eDE#XJ!aCDezhc3`4K3)>FC`&Jf>GnT$70% zk(4Xdt&1v=vsIVZsBsuxXb+%&C2YVbH3*d3qrVEtPXOB8zW;G|_q^;}Qw^)#znAwX zrA!HaWu3pW?tkNfj$gl7)_9Qfv-SFN_qX>Ra6Ic?i?6rc&mWPEypwG`>E=MJN&W`F z{N#bfU7MLDIyDO8lb~L$M(E*FU-XW>)nT#2zfTQ@kc2q!#a#d2kJQMHFPcZzoK|51 zm)d*MF*nf$b|T8(IKZLfCAUV#@|2aC?E3Riza%n)^4k*p$Q;?oJ{D2isr${lU>;wl zwLGOxo_oRP4oR9oB3{G4+3NEvE^|$=h)8y-)yF4GGX}Jvd=DrpDr)-W!Y>RA=-DwS zCzdv{s1y%^w}dj<j{Ks}&BP!a4HOUBEh()t}+?^?~1+{`MfCWhyv^Fk`%FfCs3w4vRwY{zRjj z{TJ865QUZ~@(0`#RTj~6EykTF&7{pmp0gia9+(x!m4JM(Z}KY^jvCr>ZS!Hz;y83=dVqIS)6-l{L%s zm=|ZWhrt?}vtm7b)<90m4$A2x;q<)LV~>(>uC>(Ab`=s5*>n}IuP*a|#-54f_9tbx#q?)S}U0|h;It2WKDi}7Q$YR_W`KdsQC%D8XDDn(jG<5N1O0FY|v|K zcyl86Ldah{RPwy=^Hl^&Ud4VRxr9J+4|8js(ZPB264F-T(r=-|Tem!(=nCF`m6@i$ zN+cin{&)I&ElTu)gr&~j;ltVNJjyu`M##{s3jxM+mDm?wQhCqVM>I=yNYZ9nyc!G6 zfJc7nMO3(tN+90$&Ue>u!bD543_T}y{lqv0$R;9WS(GwZ@zJc3>xhJ_=lCk{JKhkb6i)uFXok46A6yd9(ls3eZ_}w zSCWTfPzXh!C%hwvM$7~wPLlq|hzyeYqMi1exqDs}wuf;Z&}`@+*_Olj=U2f>mmmw~ z2iDQW_Qwf5ynBDAD)IT-?zmd)^NIIc^+Cg$50LcY4_(MkwQ+K=CRM=&-oM^0P zNILxZgmBa#5w($t37r=FKNlo3A0Qn@L_mgz>H^tky)icbnD8G%!=AZV=knx4bfWf5 zUIhBTSjeX51pRYuz_dC-rWM+Zr8|G7O9zf}QWiRkTZrz!zAZH&m9?Qn-1B`L2R`$a z0HfU^*y;J_)aQS$cF{>1>K>)&okXrWMjNVbpNdY%xCD1VV=6#nB8C0;V?tXrgQTV4 z*@ENB?6t(7Ks3?dln)qGSb$yF?W4k!^A$&g1UDcU`XTUkPU~y_Wg^(f9V5H4Qrt^^ z*>CJNY|6eT8gHpuWvTm}sz#;PKFNv2HeP3R$DSzPBSVyce}Yo0DvQCy`O6)*;c81Y z?OL0Zs;}k1^|!d^6gMVNE}omAceb;=>cHI zAiZOfZ98t5H1m#;e$HiqZiai}<5%vrZT$`}6Ji~*2X|0shIrZOl(2aA1y5@BW{V^5 zuo#hk)^X{JzF%wK-H0HMssSB4C)h{m^s|V8WxP2=_n$nctCOsW>I0wUCQZ><4Vs=p zmoQ4H_hSHGcptbKdyk)!Crnzt;Q_LqyrLon@H!R~lxn5F=ws&PgkA#NnyOKRE+fhK zY~Da|bSl3+WYFVlZKucLrz_3ES@l1vCvq5;)nzyj%Ma%(Q(2CcXe*^m`IJ2UAgMkL z-D5*cbHh1uIFn88Rb|UfzrV(To4p2C$H`WoR&PRI527N4Gw;hyLn!YO%f~c?GikIP zi|bjn+FW7Nrln4nhM&k>^zBUUzW)gptM&I{Lk(A}0`a>4jux6}tBDU35)Qp8SXLw( z^DyF5VH&NC;1RYJ5a#t+Hi4A+1%RP`0?cCvt_kroKxQ739GaaStPj^%Yc+clxX%8Pe+MSiFnJ*1o=s1YT0_42a$>>4BgpkZV~>~j`(RW_xO*?PHc-wvLjBu)nma_#94oczI?WUR2=BTVmu9IEwSNGO;?D@E6X zpa@=4=W|$Wh}ChNjXb|svqs!T z0;8)HPhK_zy1i5=)-3%2(2B8~lguEl4A)yI0ECu!4EJT&G;`-Jwa;^+7NMJzy>BjP zyCWkQ9EHa|!O@>LWujIGGs}s@&>4jQ2cqH+z+N?OW<8#L)dK25rhW0uyt^uVu1uEK8voP547p3VP2u#F(~?}?)&1J} zQ}1(o8~A-wrK}M-o()QI4UGhYCZ?EIo2k^A`xZy!KxJ>!@LI6krRF3?K{MDbaCNH~2r9^&^sm6R?8EQ{g!A;fMQjR*6>229FuA$ z-OyY6#c=vQ>Y<8TPEMVoRvdpT(!H0=OaU$Yl5I}fJI8&*l1FTs1ppC`-k-rsWs4r?NP8$`;jc2tV6u9$Z@-)&lY?l*MOEquqzCKnyTOOqBb4aS&Hv zCzSt-x%ZB0s{0m26+wyxkfta_ilL*3^b(LFAVqqQguqE;3j^eRN7}PBpkMs>Fi@J`!JRR zVS|cEKx#oy`_!!ndtM!JE!Hz)P^4c8hIP=hoq;!nnos)`mEJ2#=R0@32)LE= z>#ISEqF;H0j;?EO-ec&%@*Uf#W;5=5(?wIshCD(5NOif7f-|KdVJJ=JFX1E+P5E`RZq8=nr-k1>nb)OlgYV!)xI-8vYAn+Wnk9gL`ri}j#2vpd1$kkE)vhj>8PnVNIgO+G6l-s(!}1AaZw=gS4D7!n3*Ev$A}0kfSFtw5oN+(c<2&% zK{83}J@9iM{{(*D2lUUc$91(qh@o9H;q_ieAwD)L}T2yD6tlox8bG0YVd@RMv&Qeg?j0>8B zT&Ur4n#Y&6u;KhFxK@vCyX-qv&>-py@=Ef4xdY{THQC1s%B*XOn^Q2%LVCap|MC#> zg_i&jKV=IvF_51<9`?WfYA=v1JEPu6zM9gs`b~C+b3{RVeEaI4ylo5LRXaFIvhcj~ z*Mzs>v{M4cs$0|i!4F-5(sf+=`LVab*L~L0Haz zi`(P?=`3IpZykR={N}U3hg~r`u!*znZ3O_l5Z?)#<-d^Rq1qE}bDV*W~Eqv~)&Ws>fIFGXM zNwBcT)7!=YDHJ9__D>j2e4X7f$aC<26~D|Q(E4~E)esBh_{;wr00IU5@!tS%&))~8 ztRH`Vr`CXBj236ET0{$ize{l-Jn*lEKJ-7tf&%K}DmWKL@I0K?5ctXIes9>mg<1_i-R6 z?5+P94^omWei5IPZ{}A|ZC_39g@uHJ7X|E~7w`RdFNTSAb%OL*Zb1+qe@oYOom-i2 zs-X;ofE{l~*z4|!_fQB11=dSYoL}~T9X&gOHhC5nWDdP2+tj(r0Jr=0SG~E_x=C_l zyUqAL$%x#=z(e^h8K{pVG4x9%*Lc?X0Q`x^e#wW_gKQ;;Ka9h*8|m^iuXepqt_&ds zxPE_KQ6s@6cN1=1>mdxHgV!|*MRXA1_Js`tMjGbai2^8rVskp)A0?q6%q7L3_}-)O z(cB2ywDD!c)v=b~$~*Jcv$VrNfT2jDY*6q1-Wx(d9Z&85^Ll8g`2M|YSaR1d+`Bw( z>L3ow=75^QMmf(AC~=3_Y`5?**Rk7sc>=JukKD$+zn?eC3GLY7d)$4D_I~E+AG%k8 zVXVA8W#Eo2p)S;oSXKi$1t$2Ld~hs~FX0R2Ehy^@F<-jOhaCMW-n$)y;m~XU>Uz;I zUgVYgpv2JYTe&swRSAW;c_7I*K*;5zV2?Tnpzn%^$3@C|&_6=vZyZB<2wd48?>t=w z86xqlW7&jzmER3Pk~0s)(!HQ(xlhSY;vU?`f#%TSW7tC(1cj;58#g}W7nTyoZ^YfA zdn4Jd)V-%)`7PR8)p^+sWn7ERtE4ZYd@PF&Xf|d`R?WYke#CkcBy{#d{Jg^_nJ-nq z>tl}By7Z_=gFG?op`HsAn0dfARiYVMBmN*uKpTJz0(PIT2co6JsR-9ALFl{=hIX4S z#(+WE@;@K(KiN8RAId;cGrbZ7XBWV4--93IP}ZrB#nWYMTPNBETN11EndevyT9K`e}+_E2dc}y5vg0bu$azC@DP}?u!r61k&2`Z3c zsq@bC7;CYEk{_Vg3GLBo&1b1Ho+2Qt7!UeS*ph8F`SEm=L>beDap=ZcJ`D;Izdf)_ z%>foyF)z!LaAJ_2xIuS-?>9)D@EYj=EDElpeM?QmTFeO5a~U`|{PY&jb*WS09L*bkv#PJw66>i4(ngfqCmkHP#1*skAiqF{F+D z?zdk;AeiTL^qy|j4c%bNezzsyOG=oo ziV7!MhaS>FKYSQY5tKr3_GZR-IPK3)p=77+BnL>;rut2I05&TDXtBoUNkPDdJqMti zQ|mkYhgU-3;Z#5s-zY%G+Mo&Fu7=NP>ty}Lk|Jm*cBSXphZL7!I_r(HG=k^$9wtr_NqKZuiO^8k0b8HafgOS zVBtI?K%;kR`Fel$8pii$bjZXaL za`Oid&6z((gO!I)xCcCeR@MP4zwF7A{IfFLhod(n9J+aW zW}EI-5appOfCqkObN61-pojQHKcXS%fhxlTPdW$v&_ivdQF0_mMf9GA@94J5^&a}e z0ASh?P_W0cZ$0Jp)t< zgg(`2PkJG(!pZt7m5z(7>W+UI#>MIJECUE;1Heu`Zb^^A1MR@+5^*-{_`N#F#8ViC zl4sAXRvW*|eDn01`+0}V2pSL(Z;Hcre8ex4{yrW6Jr2$+rj)W_CzyVyOC1Qm%mLUh zj1|zxSgsIB|3{#23P5{3FX+G+PCFHSWPr?=fvuML7@we!|1u!| z^DNhce~<{zQrlnhxLez)Jn{6HV@2M!()K0j)P-0PjbQiXbl#{-z#SYWZ_V{u;_6X7D(0=2QD3q8-!#pNl<}3WjLk3sPFT zW6yi}|B&cSBK-QX?DOPTIxV>Pk8!pcbp4V4TPdm{6PU4+h1#BQyE#F3R{~&h>cEG8 zC|J}0+{I`04N@I@kTXGA^tU3SIK(mc0mrbtzXbcsF_wF5`$;N7 z`d%o+T=R0qhbS0t0vNyB?0V!6#xGS;b~)*tpw>JrFqC>j`;9wzzL}f*=uzmIEN)6vR`}b3qG60pH7~f9$_H~e{`be#R{$J`Jq3~{yQ^xmSTr&V9RT#Cr|;~aoj#NM*Sx^I`sJ=tpP~cWYhR)Wh5qRvGE8Ub z`->P*mqXD;Me_R7m)Idw_fDu!8!XHl3>IFE5>{4+1RGKow)6sLn78_yqNHBSMhey| zFc)%woyyZv{un0&$p5qO)|Wq^!ySK{7V8t-)|$MEfC-(POJ{@uoX^exoTO^SxBTU* zHa4J#GFH>pd`N8baV2kK`G3>{UU|XD^bm6Pd6|TP=5n~k|7Vk-@Q8oC1%BIa5%5U} z^zYC5u0f!)0nkkk^Jh>TC+MH+dtA%e!PHAf1sU{ePz6GPbO~_E&$X9(9vbv+8x=V8 zRFU@!w+(j13E148qr3aRUqIWs%93ZdDlCn|f6i%S9R_IZcr#u=0f-Hp9hp6S@!*e* z4*T#Q*5y|2r~UjtAX7(MXbM6^;O^UoZs60OAN1?h8y&cmsY6I9xiXE<~UBO?{@q zrxcVcs*AqASVHHZ4P<}r=>=tqjRS^7x6h@QV~I{vt%<5X810yj2qy*W}Oid^@E? zr2^UWmFb%vl+UjL8%K!0ruYXhp>Vt3?!tib%@+R9P&-Hg1Q?&B0ImcnfOJ=yKajw) zt^t*6lzamBpz*CebtNR6GM`fHA(Xp9DEHKuVf?f74+8-95^DdxePr^}>I%f42IF%h zwBKzPgbmSkAax4rsZM>R7qB;TbH-sgil?nX>+yqfj>GcXe( zG`nDGkNMXps)pw~rDd+Z3EepFy@#sLk^4v%_HOrl{z~O0$7iXQZ!vI}UB7+0k3uF4?rig$wZHPDsUFYBJsFvGJX_8SPyUgx7oBm65h_<;FoRw#Qz;cuZrcsG1HzplwS4Xt9P zCOqUpT?z0n6?kI++8ZyEedw-eldwF+?S=?8A_6V%!Yke3zR_6YGbBOH$^1j3l3Bc$ z^2mfHcwX{?n*3j}*g52I0wKVZO5Q4np!)!J`-!!Q64gxe0?>~Zi#?%G8v5F51!cFd z|KY+pad9X0w{9N-^AQ33jjw%%qyfZPkpMvrzki^Z3*J}oq@|Bp{OSk1Wrt3O!XezS z@disG=_ zJF4+&hO)G=^0rWH^5=ZOHxecH)hUhxgvxr>>F4li|3Ok$Ey8=)md{v-x+v z3k)`s&wF1!c`mSRR-ii;1y!fM9r z!n^uJakNg~99-|!97*}MFh?U7`(^a`1&ZuNaAc7gymh1hC`Wi5!@s-rDzmLU6s84O z-$Ns^n(~N0z)riKzR}U&%me2o|7K;(uJCE93INSHnms93xD>iu5-t*%^Otdg#k$Xh zb>BvlGD+8h|IZN@>lbqMWVxK?|38RpWr6Wj^pvxcU>0Vj8sUBO*g>|R4@V)6AwbYw z->>X{9pp|5@)h{1hmJVNQ!^DB0#?{c2Rpz6#eaQET-;CY!G zO>5|puqL1h@%d+jXdz8R`6en%Y&?=$eFZ^IU<$JY&^*5~b3 zF?%044L-n()j0k41LYiaoOI0#3ydDkcz;Xd!Nm(6S2O8>+IL%5|A z!b;r>`2I4>&p--llo$YND$Ipg{-VYZGD0tI;c>McpTM-9%tq#X!a+z;WwQ?*grK<# zgz;m^?OFYLG<7QB`QKt=^6$1My6p|VcjPe zNZkWgcR+*U$`LvT34qSY52br*?*Q2CVWT!m>;(O0+8831+JS5<`}bq{z8y}dyPpAZ zJaQ|R0s$`!z$0Ho_vCN72N?>`qe+#`_3D6;26Yp7*8ZLa1sH>4S!6=qa8Bu8%#1N0 zFY~TecW4hUq#2CWS(kE!Nw5Q4Mv5luLpg*7OTf#2exR@$6{PeJEAUDGx)aX@@-=k8 zAnt!h;l;3MCb0e^x148x*B?AYeI#P^?%2cHbtxsWqFExR%&RWQ=AI8Qr`Wabdw_cm z-4*+5DOfN6hs`bhI%G>-%m6--ct3@*s60???fx`Zl%Nek+`+5;TTvC}2s2Q*0^Gl( zUs8s!4d(_XW!X|R;ctTAa{*+944XajNVm=do7*+=$k;a=AZWxW(gUisGg1P`|B02i z`Pb%({xyxoe34eoUp<>7iVYoLwhpeFD*|2f*TD-Tx z8|Pt8G!*bp08w-d)lK?~pydD1%MxWaGKaz*KzjL6$_&2M(WuA3|DjP-`7st)konI|uch?4H79js;yJH`u z7$krLL-_pzgv)(lZy;obN7Ef?8gT%Yj}4&HJoFbc^v5iIuv>+kR3U*stlYgu1=vzn zbtATHNfhY)wZJ1Mzd3_f)=wylQY@Q`8ekt9CHB7{F`W6=$2ep@bscxJ0u+oG{161}!Y;aTP9s~H^zm%>0Z!UbogtA@x<)ry$ z8gLJH%1fi@|A5sUAO|^=km4Q+viUk+Z>;*i>{7*$w{2<{7w=jc^DIi1w*2oWZ=LE+ zMOkHko?XnRgM8f<5OoukW+?=90Uluqn{v`$>OK+tm%5Q=cn@(ZEJWRfceWJj20_$q z3=slq<2Q9-zwMA|Ujj`SE1<4P=msZ+y2*gL^`-0#6dMV#!h|tI-B$Y@1kStTZM{aG z5gqdXlj!o=WtoZ$fs(==_jllC%rLkF7eIO;xu-^^4*|h%F>(gkOA6nCf3VabT3Eb0 zal^*HEl5?YXy8$V8~h6;vFZ}@=EqyxD@J;$yCq3S#~`;a7zXT`-+Cz}*3DA<4})A- zjD0-Oalo(InD2c^&-TRB#jmVxo80V^h6?NvUTZkdc;Sx>P^hF<6v z#*EBmv|l4*Ol^Io5>{nOSrvQr!YvHkYnwl&CR&CL^w1;pT&qq}DE(!)vaDc19j2z-H04@xIHfE2MC zNT0z&vM?N3Qr#npCKRw)et}QlebS`)5DbWIL99-;cF|@Q4Uux**v=H7OM+B93J4v< zxzebqf;Fy|sxd6edQ_8+1#j}?;@8WPi1f^8A<-nilP|vQUi|9Dt7`FK7;mymn`wpX z+HQzXOjs`3@tvAFj6z^+&}v4h>BV zoX+xS{0LyD*J*N-3j6B8EniP$le)gdxE$UHRu+tQZI?NKUtLQ)&X1TC9WkI zu1(B-oa(XJT~i=XUuwMQ9p44sBZy}X3_iMHy_wqC<&CJ>xtQFQcqJFUvAtb%ZJCX! zxprkShNaCdFlbY03;lV#l~h$}w_+SZT+5vGgaOol5Fv5;=8*}_t`_GX8vZV zap!GP`LfyAU|^6I*{rD584`XV)W0BW;is}yQTAG6mnX^8Kc+hXNHkC7O#+Ai~Ib@-S*YBu@NTL1GNG-XJ0l3kYFNU-0j-_TT zPVLVJcXnh75)H>zh_Et(Ng>9WDjORdf7_0?`X~2bRIQrpc3|^nk!>(`4DkD0#O}+{ zfFk1Jl{E|R7cJi1QZ?(DfXw)1u$oG~d-YR)fQ#GKo2$XonB?wF-}hrYq~!HWXZCjS z?mz6cQJyltTS$X;;Wmvz1FfTQL+gzo?-r3vM%+0C)4pagQujGjjK0MymzM3F@n*3S zF@!iu);RzArrTy?U{-#DA^6V|`fo5Z(}L`}U7ToGpruUwwX-y;ccGU2D#Q2l?8>Y% zd2)HSIV80MOM}IM&}nwq#j5hWId~D^KA(EqX7#qJr`aE$Fk^Po#u7I zjJD9jT3fZFGWxEwp0b3_^NX9leKPJg4f*zj5$^`qv_@A%9@*GaYA;S^qPQp8b{h!A zo3n4P)v4U{QiwgJBsb=GxTy{DKIkUwLu=)mpPbc|<)Z=WsoQJ+@Z_DyDiw+~;iKDg zM5|du!ow2{3I*Fiu3nQR%hLdT>uGP)CCDGHwh67RANg5IIY!+VHb7D7Av-2EAr**f z^PH#2lUD_AT&;vo4^x@Io7(S%Mjo*O8vt*laY@68f8C<6?i1dB?b<~%Xc*)DA(yy2Nr1jXhq4H?xQNBhaB?l z4+XVzIUfTpd;7c(N4ftqGx%^={k-I~ilnqjm-~+39z*Oo-uWI*>{_b`#}7e}6mznF z&hhkm%_O#GCgI8dkwr1`E z>7OfSM;}P#kO>WT;YZZ>g{iR8VP%N_oO$K8CuBY_fbUqi3k$Xxrx|Ej_QSbGjQ)|i z$u-C??B)m2($)*6caL^OM^Y%n;-@-2JgGIHO$m1a26&--#QWW?2V9jSm+Q;p&5xSz z2&HO;XK`yJx04-9Om?*GDWPHrt4qG81jBVvutCisy>A8(zTlz&VIA#zq-UCDrG#JF z5qGFus%S+O9`|2LZEqW+MV8=07F&vw&eR{~zZlQi1o2P+b*g7t2hhl@X>lS)r?$=u& zR5J*GS8aLu+y@@*mRt&?$AK7Wp53L^)5}l(4>>ag{*yB;&(n%>jNY!WLwa|G$C@6E zqQcS{sd7qN)s9kSQ>(@6QlM<7Np(%@xIMjV*m=AMS)m|)BHfeU~-q!Zh^`w}j8_==N6FxwV0DajTrbDnERK8+7 zh37P~QfljCcXakLsc>c*g)>Y6pXJfGYl>geXBFI5ujczm>(^vC`GkuaDwO0CiS8xq zc2bg~{6@$yZwzT*eGG+aJ?rq9Y5P6Tt`Rb>d}LP(H(7qau;N-2XAjD(bqDl<^C+$O zRJ<#An;FU`Z-1kQrC$a0(=U#klIw=5W*TaDW3{;}+{^1f<&QYT>wENQa+JB(d+GPP z;J%j~^#JXDAVEC+Cx`5lTh(6D6Q0=qX7a$p5tAyHwpA=uh6-KrWob@nL&I1MXoWuy zMwOF?)@OczHdSwBR=%9arMj29vVh?VUAHJCXZb%RcP+=1g!vK|2VI?#WiQwHa8-Udv$waa9^t{AAu-Mi)V z{Mx~~ub{xa(QV}ZaBH;Ug4{`!pt~2-s=#d*9H)C}Pq!4nQj9XVz#!A3!BUT@3t(4S zY+Cs_ev+i@3EJ7Mi4D}aa(bpUvfn#G(S`7=i;$Y;xx00K8*{(cP-Cn2Wo&q&L$|SG zIs3QL@9e0bLpH$4uAxRL1XwY_H5z`7ciqBYrLaMa+Yv zTyBTNtDnW=kbU~|pxK0|Nf78;;PL5%Gje6M*b|+81`HqGo`^T}AQTnULT#CrXETG% zZSt?i471Txvznd+K-o1wZ5cqRKlwEgOqNY>r4D#Blj8j1mMSmklbL2#l?Hki&F!po z)q!c`*#x3h*B7_Olj20*K|5z~wb*AZ=gr-k8GzEWG+Qd*;;Zw|0_Mlr(<{sp)@@K!^i|jyd}_+W z>|317b7nIvM!!Agev z8?BsD^5?3wsNAekq_y!5wufNDm785P{ZnNEII@qBG5N&w8oq&9k&L=;*N#g>l2*Eeb zoQZh|`W)E>0?PmNaX~IZLx#vi6qzHrzAjkIz-THBTvqt@;K)hyB8Rtr>g1YUUD1!Iv7R!S_F^%H=?`N?2&k(l;DR<(bPG#bEi7uCFB;KMqWMJ%(s{K=msKpC*>n6WT)lE<7)n@MWmY8n61Q*kAvKr5u?i2T;aNYhCx_thxGOFMKMGt(8` zei2E-zTmG&`W%edJh-oOE~@2{0o3gEWJ>Ph%jee_(onmbo}hsaX(+gQ$K|-48)n$8 zBt;^=zt9BRb?=bGw3eZpUFvAawr3w{Li)N^E>99*>Qvdu!AG&6@8p-L-fm^FDbStF z;-KBJnF1&$@dEUvBVNf>o=m@OKu!$APUnGsasAB*hgeOI^7=`UZ54&AJ9z9P?x>gD zg8mItHa_5R$1D|GbduQg#S%v~`E`6V(O~`pR(e>I7*}!%pRK2M;h1K!JweLA^WmHb z%xYtMdM5jzwE=#xxmEg|h=Ns2ogsg+LmBPqr{}-QE%cftUW{%ls$gPL^3uWV^}f>&CWy87o3t7(ecgQ=Gh9&IMcWOar+{p za{p1qxwCbIi$~=@tk%B}u1{a*A=y1l=X6cGSY8*u?DnCs67A}^`ck&|OBHrkZ##R8 zyei8=bASYR^J)Dq+|v-rJ6#U%7P_K3-%K36)2jokk8vvjxV>eJz;O#&U;E#|BD9`hpoRc-COe%gc<@%7fzjo}arh_u!=0&G~Zf z*Lf-i!EsxS6M4Iu^R;KnEJzkLZ84wvKC6%T)%ytCP$s1;WAVEUO}6n{N>M6iW2a9> zC7JTPtn~zqf^NHVjw}S=-fC^{Q@%L`+7QTap@P8GnFSFB)Z)Mq+3($-xcqGWW<%#| zabFW{a5q6Spax&r|L;yKkZObci;;zc5e=J4i;#xHyf0rzSf1qjb&tey7$W^nfyzbPp{x)pgicbs7j*ES9-L%#@&Xt%=U zF}ElTbeEk?vJaRUePT1Xth|4yS>hFu6LYP<(;v zUGnLM2t{t1o1mA{*3iOQaY?y|Cd_Jy*n>z9%WV2>mO3Jqg0lg>fWOlK-qpxc!=-uKmX+xd zu|nw86&a~naQ)S!$v!H;z+i21F}bJZ6wR~oot1~G1-*0GEH1M0KAGB9fo&}b?%I3j{K(bI-F;hU zc;%O9 zDc70M14keAX)@jy*_PN;^Nb$0BSqe;XhZ9+tvg!_F~qgbnlu)vyie}JXoZa!BNxr; zqP1!pWAD7`QGt8rN+R$j1QQe1rpP{xFmqY-FskQ$%}->{JmXJ+E3f)GV?rWs=ibC3 zV1sNUsabi!=j8Lb7&TSlpZrTJ+eO+m^%A!&mpfPtl`2f@cZDul4TCt~=S~jX+b6b? zxb~DTit4%GJh{ztbuo8e{p@Uf%GP-FbZeNg6bkcn%ZV#Su%Mbb8}!n0{yw!8v|8+= zS!bi`gzXO8LO*zY>CbsiGo4GG=Bt^v1sw;@zLh%q{E8?nuF-y@HA$~`Cal{92@kqi z7O$=fYAS3)H3Y_pu8 z5g_GB_b9jjY(fU^qtT3ajOI@N5#Jb^C+RKylkWv$RzE)yp~BfY!5*n4l}K|vH?}n2 zMOsg|A=BMff`%@xPW;kA^~B6miCQpa6|xJp{@lr^cRN<_RY}yQo>j#xeT0ZNjc`tn zxL}6MmankU@hGh)D<(JPIY}o`=wQKyOgG^M)Kw+J4lPEO$H_fuTEu!6jVtMysyfkH z63Nx`?Wzr9S9MxK+@#49j^zqVNsa4j81l^G#r+#CFGRvqhu5v&c5C&?HU>w52}k9* zeaY@03Z0<;Drb@m1Wy}ZxD^h1#II;RBbr;-9YjHtp?G}*L+vov?jiC zP?cs?TGN6>&UtMSYPX$cF|S%P8-indv|>^@y4oBqo)cJ;L~;=lf=3VL2AitRxuJ-&1H|3d1BqqJ zuP?UJHDKYDh=Frj(msRUx<%%^(gDN4O%iLwcG1o7JIF+xbi-V4&iNQl{V^B+pe?Oo zC(L2=?~?~3n}5$g2hS9faaxS#$mQw9nG-)XdZ+M+SxNtOVRx>LSpiY`>uRtkD&N@c zzOvJnA*q<`p5Pfu;-7$7R0=A*Dxkg4c+@5CyRgN@m+_qrZ%lH%b2^HD?!>7zPgl0f zVWuB9ExGhh$F&+%LOBKZYUvg19n)Ihm>ES z042wyFZKTKmrdh})Dk&$Ps>h4lZF&UY&SdbU_G#BNXedWNu2rB$$r!8x3?J3)t?K3_xTq?y6tNAH=s65IP36+$u?OgDST z#L+-+rDbt>H)VNFGm;6v$$65M6jO2NHDpRqlP<|+x}!Ynx=(SyED!Qv+~|KDJ5_Sdf>0BtMoo9h0oX%{W7A zNaL{vCg!1(`h0rCIe)5PC2c)BkrAw6`&t$fl&G zRKW(vd*os{f49%I!b581jdR6inn>?S2?6 z==scvIGT~LAuNAPBbmkA0F34hY)WUmt7*6zH8l@@TH<4PwdFg#EUW5I89M!L9v5X`#yLNe(VkTerL-uGzDP)s% z?Z3{Uek~G1kZ6?@mrEUNWck?5)9?KJRVgNGZ=Nil zRdzaTIA<_VmO|MDK}idX?FHDF%Oap2W7}v9HytFRXu)+EHq&B1CuPWW10}|hxCh&6% zPMdn^7kXj!!2#%=auqV@>DY-+JPcHay4puKGV;wrfwe?ie2}MijSg%^Xyml}{oFEV!=R~jPFK5>nb2%l;*z!6=I z%iK2_K87*A*^us>kW@vO+c_06w>D(j;UCu_w7KmA6|Lqx!3-&{YR|8gV*Z_tS90A* z2BTZ(iM}!H=&b$6-NnyiHp2+xIqj$g(2iar>Rx8{d)gSx<}RQd{}la6Wa8Daf0 zkj&t^d{BLM{(xyT5O4;uK%0^A>Vbs0uDDiO6{h0YkBKHeA6m+Xf0-=b6$*fx=%_v; zWDaZ5PzT>U8?Z0Fu`?85fX1$$8*zH;{jj{YFda$uA_t3xbvv|h`W9ih60JymprI=_ z`l8wuRjk5-5hrP*tWwaMt>1>Xe+*+=SKd8#XW3g*wP~xTx@~I-@3Lx!!*}kKOs>gb zdUj6gjDc+HrO~{-B7v+z%IW7pb;ay^sq&UZr_6G@qS7ymcsg?XpFC3BpvS!djs^mL z+}m|EEmZy7|GB+g*x$d#a1iH?cS$h{GvaCLfArZc3^vl$pDZjNXUZt?YsHr=Ph(B` zoH>!d6YwqCFnOJD;0X|CsY2_S{FK@#a5i=cDk06jmWj8oNnz3PG{v_$ zG$*%N*dImX6D+t39C)Suf1rQlqb$QhI*I#bFh?2~6I5Sg@h%{t?t~_D9*H3IHcj;7 z9vP{&8|KQ8nsIR*f{WWJO{{jXe^h??gZx-S!RnUh662(iO2fm#Omuv)HLj3_T0i?h zwu1k%L(uai46vw7bQ zJLhhPSSig2vLWmR)<7lE-gQ}Sjh)WSDjs$m%vkWZTJnX@GM2mB^(NXD4t#E%$X{AJs2O}~x&`qg6M2E2n|QPJfS282M#|Fd$*1LuFIchL z{X|PQ@T6xhpQ1B6QDS#zs$&%QsM~2>9$ysvp7zQP$rPX8aWUScY7@579cXti_0)-% zdI2j6W{N2u7EGpRI%_-q*S_S~c5>pI$;-(}v$V40Jhu_pd3n#OPMLC$A}a}2EDXGV zrl6E6d(if_ev^Un<(Dsn{ZrOKTt76+$4L)-n>w2AeqbCmd8KxwmkD05jR53&l%+*?KIq%ODxP^63oO?v5^I-sjS3BN*US5zV^xGv~Q=&Qc&Kd2i z7Kb9->_P;JLlVl_stuAvKWtCj*#<_O;j|w2o)^BFu~$NQq`$=AOyD7}tt^)^3-Po; z-ooQQ?eTILP>fY zENxeKRV51Jmn^Zp%t))N(l%PKlABar2C~7)nOqn`$TQ9gLIJ=zezqga$6O_JJo_Q{ zlOu8b`ODmNcA#nOxX&y*bsJ#KKpzdrNm*WFyws~w`p{w{FhgWte9$Y6+|3Sq6wg)x zb3vQVb{GazDlyZ!lAx60F{P7tIrYdge&{@M$ogFSZL%*2X+C&{F6xQkT$OKrK}{hL zp!KoTPjjm5HrgV=a1HA)Ew?RE+Wz%3JP#%h-30lSMCEiH(hEl38J~Y9SlUQZphK63 z(mjbyja8?uDyIY4HMNaBcZ_gY58i%SQAcz6nE@F>rrDxfNxYgtra7be>h}GG@r?AK z(bL)y1nmo`OrtW{As#>kLP$|dP{8yxd|pA<_M$r}D?R9W9_rT(DM@AF(W&#u>Fx=O z{E5Q;36+JIR);xl!EKs&M*rSsV-m1ErmDSLYqRDYiShSx+h7A0A!F^%TWIrDOLpM3 zg2VV7dQjPlB=>a~>Mm=am~?0y#xK-RWo;R#bvGG;2v20&$~l9L=ZBj>L?+VzwogLz z0{<2`T}eisXrg++5KVX^=P#*+`jK%brn|i-t=WzfEj6u@ZXI*aAmD23i^<}GXWO*y z(~21arUne5oAF}#J#G5S*N&+K`)E8gaDWDp^n&E$mEL?8SOzjqF&q2-)Q-?2F@Wc* zp$6;7t}f66J8|`_5Kn(!a6fn8+&RQtgJcxQEE-rLIMHi@i(xtGy>*INUPT_{TT*iO{q*A^|HP@k`;tY`Ym#KnH; zWBtijRND*Zv13Fqh0A{VZ4sCaUm$k8xVCfb{FV88TShbmtKbmttLThMp-R_8cmU9m zsYfCPh8MJOYFO>6FeB8Mg@0k$fTug_<#iJu(?T)d#3x>&Sv*$p{1(0?Y<34sD@Iz-1vg$*PpyzFb5NrCJ4nUVmkD3s$0TW3q#Ajl5=R~w&0CpyOoDo zcy?M|BC!CCfw6>MB+U z@s5IUFTF9of#hsQMF!)=L}$8Qf+0=(_69QnX>l-rm9dGE7X!0{v6{T5_+Ev$*~={! zOqTrdaVE|D>f0hr#g$unH{x+Q#D<`evD;$6-b+Qz-7Y?)s;dpV)&{R*?4Q+;_V>TR zzgeQhP`s7(u)RDOmu#2m`_oo_SgYgQ8eN1(S|e6E@5+lU$q^eP!<2Ot8Oz;Xps=QT zM|cFVI=RZt3&aFGAL1Ol6_6NXRaaJ*Wu0FwrMqUTS!RuK@VA)`&pHYWTg-8zTHBDQ zyW&P%6ra!WpY(SNTBctc)#6tp<&86<0;$Qt_>!$iH{zJb1t#nI7Zc7TA?r3 zTFEvYH63!el@w#MVYGy&qefxK&b%jA8&6Ptt|Dpa`8wGl)!~FFN@D?7!g*{wA+ST+ zpTK@l#93DVG;Y$%=bFu4cKLG&R9(&AgXf>VYx*Nn6DOw-qf+zvhe58 z&xr5>(^e$*O)zG$ox&an5|MOVp&bQPO(L0Eu`a@mM;ya zEdzNhHc>mPusTl1Y~N`$>fOi1)+#0N!sP)MKINd+WUKu@z_ILLR8Bv!t%!vxJKzl# zbXMn3olwK&Z)dnwCyrT&9LOI26hN%c3nK{2yGp03(qk3kbDt49$EFHmUo+%w+laX+ zR-=Tn;*^SrGyWtUaP1;fEFduXe(Yk^Rs-4EH9zs^dg*fOsm0h+BM9;7Z@Rpuvqnt% z!ExRe*i1nf&G=E})$R?Im%0UkM`v3Tz)6BiO7&c`*0C*^_jc17f19zMxY_!FYeVWi z549+&+1h#dj%@EynUi#8sFbGrQR&DUO+3iJvD+bgUPmVwsO}@szMBvajO)06IHFkD zVyuMLdM4>;+wMDbRUN#rWiaMivESH~km|fMXeDLTq~8GZ^fnoOWF!uwEqlM{Zk-$* z2~IlMdFl!gYw?RWF7$4O(Q*q3zn)FTj)hC$24mRTK>Evc#`GY`P5^b-R9Wb`-Tc6@ zwkE};V0O>jFa@LE_wm)Cy1Kc7ds?(gYA^A-XZsiA{oeBWJ0|KNE3fvTSFIckTT~e| zFzYtRUKiju*knG}*sl|5AScFo;@Xw7GVxLx$(q*47=_5A3*rnxj+N2XmviH_BNy_W zd+;9Y8FP7pT_j?0-E^F(g>%$^&A_%T|K|4mBZF@7hNiKc__yD2MBn|cX?^bda*?^_ zCK_SlkrEcQE)%b}aZyxeW>q_2c92-Z_raRxEk(5R6EBV%SKfSB<4`y{Bfi+)G~AO3 zWge@;6LD9%oKv*IEF$kku`83B1~r=;S96`UYb$y@eZRLrB`uNV0P=ivPhOrs+P(|a{IQMHu&3){Y zS;5^(1`(*NY>84<7S_a|kz8Ygv^g0mkx@6zogo8^f%R%PZ=_eo)t-xtb``lcb(Tm> zf0Gv#o}dY8@Hmb;?cIjzzW4i9pUt;Vwc_n>b#aAO&;>>#s7y_PO>0Ig56+9G>Iu&A z1RYYVr?Yj;A~$ejy8{xSoo9o0iqQoYr}f9++00i}#(ULbsqCPlDtKp8(xajt=FxC= z-tt*^;vJzX=q_8>t25l6Sod!1tE)RbAZ{C5b-oQmzgTli{%@wkfgg}KQt3FgL`su&i#(rvR7Klj* zA0<8hkq5WT+|Azg0#_ft1YHJ0-~KP_MYHIB=PIh29;ljX-hOpVH!cWrEpLJv%$cPB zZInlw*w@6b{{YuGzX=a%0jA~6gQPh1fA|2n@;pqR`1Y+XtpAuJXrYq>QDab!B0Zoe zZ&)0T`|fpXVDIiK+#M>szKGy-JoHsX&0VnluIo{Ase=nSbKcEs=1>(`du<^C69Uy| z4S{(SZdG5TBK6OGWOW>yBE9F2QMwL|`zr&pc4ffSw29`|31~Qkfy5|C%>;m3LEl)I zTrz=7ZkJk7F4DnoLHrN0g3yJ;LINnVO>iJi-3CYULm-_sIQt+PI6}4Hd>W@PTY3W& z2~1zGd(k2XYMkbwL0_O8>aDqzW1zvZHrzaLAR4eFxoTJ*F~IEy8jQphjc3 z-^9=^C87&RXaS_>SpSE$|CaR9Q%ZU_g>DQ#&eQe(`4~0~=JR@|c~m}byllTBZEZ?q z$12N-E}q^3(}8fc*Z6Jr?UU1k8jOR!(g|PKZEvde;nz|}?~WgV3N}1nrZ`pVoJsi* z43z@RgR-GYP)ODerqP5zm4JG)T~VAFp0z_B!>$EUE0N=Rt@qDsqF10lRo(iy$`s~CXbmWkKdT0MHxTcF>f&_OGe?^A; zEC}J%@mBZI$jjsY{kQr2NByVj;Xm9;%A7jH>(ayPLBHpDFgdyo^3;;7-HN2dXk?(a zva*YXxzUxN^LJ}A1`Lw$nC{N59k6z7n7TbC+0G_y0ho#t6v z0aZ!TIFS4TK!;a<&q(pwoBRKt=B_*(>h*7zQc_upgo+Xsp_46H+q50~7P6!;_OeX~ zNu5p#QAlLT8dzG?pXdH; z_kBiT5sG^DzY!f1xnNopuA!>EM_N^1#uc&^pglOp4j1?40}vl?}NFiR(a`dF4*gGhh{I_ zUq~tUf316DH74BWK+rl`$*S28!&LHn+%4v9&$F-VW}K*3*)5T;Sy~B>B~)wL_9|WV zO$t{03guPA8&C2_l5X6|pI?B<&jty^x{&h@r#X@)8$Ydz(N~HUnffF^FD9X(6|16M zd?_#ce5zuS%2wyF5~W9if6hWWJa12>HQTeM7a;VUUz_Yn>5Fjm=ax=(_Z#?pmWv|o z(0!`@LBB-4_GWLj^f%VjWo6+zT7T=*XK2FwS~6e!%9;cpsg*kmHPNo{{(|bpceCgh zQtk++X+}3ecIhS@o>;lrud%2u77yhuU3=c(2K;bG4&LN09`vit88;bd0N-|0T@Sfx$k-R;G!Oh50WPz?$_ z?S37T21K9xRp+_V_cXwikIM^@XF*WC^~&4reIi$y8MJstC%tQ6f3*@O#PVuDSk)Zj zUw+}~BNQzm7RcX}3{QncI6a#OAt|B=rCE4|xFG~RGwckh5q8Bh=HT4V4TYA>$EOyN+J*-mtL2&r+)hwX zj61PKFV2CLk7X9XR2&1`T`Nu)qZS9>9ngliW9_Wy!mhTRaECFOHPPCd6%V1VX$xac z5Jf953wo{1EIk!}T_LcchX8c%?&sP3ZZ`M;5iZ^~*RO9*>IB10h^V|6mXC%XJaIo$ z*y*x(hPUDDsD3)SS~bws`t&4pFTRb@=Y6xOu(g=$v|H&HD ze>83so#agru$VOV6#Xpm5V~}iFcgd8r=F;Tvf5dE*AZLuls0yvkW&A)(0Z)fT>ej>u=V1%@o~iR)fJUHM78>jG}^n{5ny* zF^O|r^S$G=zfbTsoii$TwFZxmqslcyqr_LeP!xv&gLtlxU7q13RD|Zc|5-A0s(Kf%<#)H^C!+asDl4~`*Cu|S{GNV`D)Cg+Ph%%MGxF*-O7L9s(~h~ z!tF$~3Msv5WueNj8>GC;chTaoao@r)W-<lK$}Ddps4=6|dmVvtNX2Q=6xE z1TqyU{iQ+qpkiO^r;BL)&sLI4>{{Gu7 zzirZWpf4rI!PX?FweIHO9SbSoq)2aHhx~#`XeggL1nW7#4JU_ZftgPczpXs*JS}2! z>ZfkC-KT^`q|aHNNIzGR@Z{8)E>K;_0Ic+3&4#Tez*ZlH>fD%4GjFO(dI#ID*<-jl zJ*w(J*#=y#5_j0Rwqkbusu*C4N01RDe%oFA!A#(wM7R~9k^tH9KZOg1G85(^5AM8= zI*3F&W-)1U) z4D1Am_W;2Tz+69|jF9L(vD6DStB;(vTg@fedFDX%;$isgIghssgDik$k+08g@9K|H z1Bz9rSjC*-eTfluxHc;xfQawg)Y+P9=@A=(?k=iZ04#E|>242CX{N&mE7FKt>b3Cv zK`vpepi6wX?Rf*~vjH&w)Eo;02~4}$f;x;EVZC~I*~(dr;Or+aqtK?3auw? zfIWIKS=ZOe5)7#2(Q5(V?}1;{MO#%#BcAiOZYF_kH=f{7)1~NT=d6D zuL)?@zjM&&u}|Ae5;`9FVUb@{l`vHQQ>{8v2Ri{U;9uC`U`nRBSw9>|E`PIO*=+Nf=e zrZfty+sPOvouj9N5EmJX3d|T`(?8D`5RjZ`@o6cSXP-umM+kO)F1w8f(_#S!AryL6 zOabRb%L{V|odn)2*r?5G&|ESIDK4@%J!sF@grg=i7@SHa52SH}*?BcE$c!ig)kJQ= zdh6;3uK;0KgA`B^WIi0t-aH0~k2*qfqkWvz1EQxi}YS zqrihVXUi-*dfiWs)RLNdGoJM;9g4f0p++k{gG2mUFGv#$s1b{$WI9u?GQu7p^_l%( z5NnI$ggs6s3fQQN(E+J+4l-#VG7DEkh#BOK-CID44}Y-){q9sZ=^FO!|*774wOY)F932&4)hU+&l!{J&w=+&}%;i<%AL_ zU9*KgCLD5*R1WvYb?a)QGIFvD_)jg+@nr&1tuMmNAHQ!pcX4Al^ys)LFM{`w$C|iV zSv*iU;}spQLF25pa!S~2!Br5{AwuJqqicG0-5I;TtkTzy($jgPh}97kgE_Gk?9BoH zZVga3?KN!>GoZE|)fmwjAMggD&r=oYAovhauwel;_82dWhng9rUGIVxwXZiGgc`Y7 zwqk>Ml>E0@mdp`GhG16q&RN}9Cy)UtsSX`O=f=z#X!*&2rnE|Vh~;I@aJjlc@pTJ8 zRG#YQvT#TOaC02+vrw{s*qc$tlm%E>K{}kwa7h5n%3)#ZUgLwk2A8ETV9oRQ+{}fj zO`&Zfi zsqG15#v6w~!qyX#RSf7>xYBr*0KSIItN`MJ6yX1Ge?4W9s-K*6{DAB-_TeVG9Q;%3$8|) z-NQcADkWu5S_!2Ftt0~07=PjR_G=pT^!e{Ipn?38klP}x=_ z4F_uIug$7XozO>9`#$|@fT930*3bWWQ-9ro&{cECLYsM*ok^NAwI$j~`}5I=Q;f$~ zR-&f6G=N)hE0MaTJBc8CA4ZA>k^Cl`as$MnSfe&@-HpihC>!<&V(!S#l^( z$wQFkFgUhSzTyjC-b%HR+12FW4k!48%zv#FN_+Ww3rInH=)R^Dor)%={hud>dfLIf z5>wAM>F|1^u94yVL3mqbdf^B42#tX{aa@7y0_ubuPe~q`x*U%bh}qaYC{xXbT=eRw zQ^LUX4u89a;dt8Z-Tg;S0?s_qcuicB4CRFn<{8ut&nSQ^I`tU%Q82gB_3+_e#{(cM zS9<? zHBt>I0pGp--A9dQ-NWk+gHgakBU6vk-GRubZlBcK;vl(t;S`6VQ=ox}uv{nK{djcq ziu~INUWyNsGR1QxnRFgIUVK(g&t@VE{U0{q5$3F7IuyO%=d5T^!@_)@0v`HWFM{r- zNh|g}vs+70q+khnHEp2s5<33@8eHEK8|lysBbW(!4PnvX@|+8rvobqi1OZU*b;q=W8#RwaOU_xSSIP=;aaRHY7$ zwJ89h67I#hHuJ|JxA~o`hf71Uca6ez`Eu>{{&hsIQt3u5&AcHeuNp95ol5t%s!uJ% zymuD#8(y0n-3wmOh=@CUdD-P}H&ezWQwjI#dZ{_E+Rl7x(0}SIGu5D(>b1BapSN#% z7PQh^5t)D2Ju(0O@oRpkKoKHZC`u%U{D3)AkO+PV(PF|^(Hvcx_5Z6)S#uOc#@VE) zd}oZz&5^sUds;Xm)y?b)F3Bq3}RTC8ROu~oT@ShLE{YM7A8D#(Konoi@p6p z`_v>Enl7soA{@@t%PCDIpFYxrq{@vvrnJMMnZq_%TjvV1RvqOSkK5Os*gM5aZqN|> zQiO7g!w7@)4#CV^5;-IT!nFl+_aU>NW#`lnEuGkUS%KA=K(V00QI0~#%U{YKd^Dbc z&W3%~!uMZ6(axtEF{sfGZ$-aFs>Ng#))LL+6Tz6d>qs~E3V8zQd3R0bYZ>uLrN{*? zz_e$9U)+KGmO3Gpl<1^7-Lwf9KrS-wBtSm-*$;j{cqbJ!CllQNf=le`*z8pw#Sk>Z zgVY($KvrAseX0zCC0RzTG~T#kb`)dQxm|?e*!~)5_gt_Sh_384@n7MJz7l&ZTsT%l zS4+`3zds>G=oAXSNijM@lb)(!4@8~qK6s&4<=9i29K0*(RwC^AmWoYU5&c-#nKS|l zu56!rr$c2s>ZZ}0vc}AqrWkQC=_J!QFu$Hfd)H%G%(}#-u{ig!S@J3u05bjH3~_*k zK%fRJ7lYgW85h0*qYk`%|4k%4Uc*8BatyLO|5Z=o9l02`I#+JaCbL4I1mRru0997j z+yvWnnCoA>0?5SxlOIp=Q`)71yn{n%qR%cn(X=EKVQ$mUWGmFg|O{6E)6l&z+76t3wuQ zFN)#1`7KX8!jLV1;4Epbv&BhsJR`0t89*>Ks*48K_fLcSDJkTOl0v&@cGDNn4hXq_ z@?L8LC;jxtUYNs%eGooix`l?y1#Ev-?f3XuPBGR0<-H<0(=l3QLOx?^i1b0wxX>s z%wmV;Cr60zWlA)kuP@rfKn!U!lbJ<+so{2Xn&KdEht?c;TQT9>wY!9opABhfQM3>r zZ(p=|{C@2uMW=D^%C$lgm#35Mya83V0I0ZvxCn^gC4pP?sEl4IJWt%nRc>Lri-cm) z5S_AzAn)^yoCkM{RDfe(0TdQE=m#!CAtyY5P+Q16QG-_rkFAmq5#I86*--Q6$kuj% zoSjw4OqD&r-4umaHq)b@6P!!Hpi2UJXk`?7Zo=a^9aC}f4u41nq$Gqt8rT;vF1TZb zRzOJMNGJW4bFWaM2Ab6sqxH}skVc0f5F*zn7K4jJ?>}XQA1Y;^TH+D$ph8*dwNoOW zJgG$m8Bc%nhRP&;J9N3gjge?`3lb_Jwj~r24kasQ7|(cut72w3`5CZv7JumZx+QTi^p`F#=kc1)oPp1xjy}t8=(E&5T0!BU=w5W+@GJ$8 zx-N~w1{N+AHT>VB6DUnE%k?^Z&PdN}vYgxgQn19#7A=#?N592?U8xk0xq0ar9TE^W z5MewJ5($co8f=&L8-fEJ4;NrdRS7tchMwvy*=&BRg-`HF66R6kK(&5)0*YpeSM1Mv z*bDiEI)JFvb%20%e3jfoNG%tcLp^;qN?z`t_ab+Cm4qvR>k;~NE+PaRfG2g3$x?^x z-IMlfwiOUA9exn{lrBq(HmG~;@#uJ2hWL52pVFPN2Gd6gHZn!xNi*DBezi1d@v{H-SBzCSU7%EBbW{yl> zSBynQ(G*#$r@v7Hh~=cM8oEFsm6q^&iC|QpN|G^IH{GL{WLnUpc5mUYaYpk(x3(Ie{?{OXE-uSa1a2IrO%p;nR{1Q1L#s|6%07^ zg`wc<)4Lx;{EXfD4P5N6b^ zVXAD>*pDa`C&B@~3i%T?*$(m^h(Xt5B=$GS#!En1I})ri^O{lIeHW);on38D&lbPY zY`~N4C=qEOK2VRi2$Go(L9Rxd)2?t#>8^-jE+#I)eKcrzk<(CMWQRlds+ig&dIP}d zh6l7Pg%Up~ChdreQ%s^u1Z!$+=1^Z+(u75U5FmIN%NhyuRxCnz=IjL&W+{jG$JY`r z{KZp-kOj$m%s4@dMML!ojZ8g3O3b$E0E)f=>#+Cg$8v!U15=k{k&A*{C`7E$e$0RJ z(@r@~PY6ux7rJ2YU4s%Hl5H41cbFSOedp8)e!uo1NL#zB1$ zrqhhO-omUJCxb_K;K}x5L0Sp>5*~fA=%;M(vjgwu5fJtwtIC9|s_z7%2i%po? z;1hRyU2r!2wy3tsf_xmL{W9kS^Zz2mH~NGoD}yn4IoHw2HwpPcr>NLZmO0&($9Nwh z$UqGwVt$8g$Zw(=d}_;09cxVZ&|;zCmCE*=H%h{KO4D&+wOb^+%#|9xMuvktcPGSs z1+oxBikVA(XM)nXflaQ2^m^4IKL)=Yp500hlvKlV(Iv14o6gm;Z0rU&{fZM;Bg8FSK6zR-H+1D@x>S^09Q@aP`7T`%ctJ+IWXlsG9^U59*+Mq& zSJ{j)KkkPcd%a#XJJ$d32Z4yoN2O*p`9x!{#}nX^B*FT<3=Auo?-9Mo(@c}2Fcn05p%x#Yy^5~yH-|Aw^zgH-+D9pDM+Zjsw> z2_YuGiVxhYx!6sPE9YE$GDjZla|hPsa~F;B8AU7un_VU`T<_cAl- z&>aa<-d{6xC+^@&3JFgHx`Qi6$}vs}6`G;BBM@9gZtR=EnsB%!I-k~til`-LBPJ^0 z{N}{7j&Im?&p8FCAsVFiK8Sp^#e1NbAi8AABsNpE5T!4+(X8T0D?Y54IP1|~C5(8OFsBz-n*NXB| z-_TDvNp$)57J#wga%jikI2m1qY^zaGBVltdkacm(jJ{fR_d}TrIS^SFS}~IyjC?q) zQ-yH2tyd*g`U6$+kw!=!AWvx$n0s?557$le8QOM(Y&J&=M5G6I-0RkT78niDzC2>d z;=*tpRv}7vHA>J&q}@Z;eZDc6t+~OM^|<@Eyn3s90#zSbD)i=OkO&c5&ZTMjx8rXVh!VkM?&|k(< zH5##)D~3{I<1hZn<3R3EX>l3xaaZ4a=O>plf(8d-E`yt@2B>7xclkO=8r$Rly}FPx zJz;NKhtkIXpQ;Nf9+6qrs*j}p>vg5|>Yc5k0$)2(W=(&s(C<|DWM=bL*{hcq7=vRs z)q3?L9o)+ZZp(e3i9cg9y40w%;s>Z#xX!0XjY#~*7e#->(gzg&SKeqMxe&+){_TtJ zZsfK(liixq6fBE(HEu+7i8PPy*Qa??YZE~;2IKzzJ8}Ep9>{Zh z?oCTKlcbh_|9xt`Z_s33^hclEt@^IB`19xE6-9)~hSmWV#Hd72tSB5h^l;UGED-#K zx5}n()r~P4I)7h5+9UV+qkyQc@uEo=^R|>jZZ=5n3;dvT0kA~*6kb3D% z5h7n0EI^cSJhSVEg3B-Kco>P-ApWx_H_zx%vs^zfAoW+Fk`_W!#sM>>YnY%I`Aw@w z%Tj^;-+uM=Pn55hBf7uy@Bca}u57T>r+CE!Bviuw_5j+qd!tcphAw`JvtY_;M~K!d zkB7?N<#2xfi)b@s^N*aoi7Sl1w9HPRzV9VobYh8Ee(prCk`2D6zbh>Kx^rrXKdem& zUMP%6D}(rDt{__W75x?j-8@E2=a!wVcXz0R?{IxYxYA1}eIa)1&ztJUAVKAdCa6H% zt@Ys_4AL9K2um5B3!35y2UP(J@`ba@z9$1>#a-GoWmkfhoWH-nZ?`(EG?FQ5I%Kx2 zHHyAYJ*}3B7f2&na9OP+t4n@YX!~wLU;(^CmY7l7#zTUBs5}2Uq4&MvxMH4NY6`I6 zH&au-^o?BdmtiTAe=;<~>E^iQ4UOrNp&9bHo4UNAk)jUW^LC62S$ci%8vh%=?zwMo zs@V698`|;IzJIVEY^Sd)VgjwgZNIA*{Ichw(P4l%t#Qrlg{)=ikHsR-_=eqAkx;8&K6rgOH;y2QCZP0n*ixYL&_Hf0JMbD0%jcvDv2EoGyQ% zOyaQ{@n{zt@x9oKdnT~mBz*g5VbMhG)?EJz|Ht@*uxrXNCCqNUhHl;Ml0>#u zHg^?DO!zt%%i{~W|ID+Q-S;-9bNl>_qA8>5!V(_&(5eHCFL6CPMO(Z&;l0OV9NRuw zu4zMg68rfP>)fFew<|5hlx1pgkdAaG5+hwXw#g<|9V95?Nq@4E2Hw?*5N5n0p)k=? z7QgV0l{9W3H}O#h!^C~C>?-MYtWwKads3wg`Fg7@sk964|F-awXz^IE0cnV0kt=P% z)#-AW#P9xjD$>=0C{<7;F)@v+xFU)lZ7z1||2SRG5^fQ41UEP)ig!$h$9P2TlpL|~ zu_13Z!dV#ghhf_~vRdbl31lclPWo7kYq7&7G!W$z)d%6zmb(Z6geGab=P^iFi|-CkT)N%C5+ zY;!wuE}uZwsxCaXc;JC>)~M)E#zaSkd(?bGYS(?Dd8Qc$<<|Bq3*0W`F1d+?LgMeZ zdDD^%KGErP!c@y15yBDt1|0jAsK5Ld7F~$@amj*=UDLbN8dhlQW9-+n(FgNtZrikS ztz!IjHo*s48NaJrE3#}mbA4^k5e6T{Bi!rOFrE?)DLwcu?#5=!QByV3r%DeF<|tY| zx`^3M%Edy1(}T@&AH0fQIJgbxl@wL$4-RK!){(BUJBaPc&M78K=H{gc+LA+eN?p7b zVun?H;mUu~rq5e{F}MD$x!z*rK1r`CR{YqBckKS}Ggp%1TVCg!w4Lj-n0FTK$$jBX zPHY<-*_LA)b*ufZD(*$ZS`FK={8(I1{rzys_)+zD%5H_h8@*n}vxi1y zj5Djw3<@w+XR~_L52gC;v7rp#cXv6L;8&@V)f29At~Kqx(ALKnTq$+7Z+ZoV2(ERD z#QI|gZL}7h+~+8Q2DyCQk#76Tl$cND2D3NfNIq6xgqx-7DtI1c1f@3O+zj6N@jT4< zO?us;eGmIZ$2!~g=nV#4X?jz-BA*@B859(?>DaAn?ooK0u`X}jV%V(BEqTqTREv5s z!ATpp@=U`$1nj!JLQhMb1LE~wx*is{ic^XmXCgIq3U_uZ$p)EkK6%MahEwp;o@1O9 z`a3V`*>||NrJjBVpZ|42cMi_iLju=LxVVsiKG$E9;AVGa@MTMCqUos5Gf7(~?vl0H z9Bgf?+Sa!TwTUqj*@>LQeMI3~q=%%YkCYrqd*^pkBY~#%Z_V=L^an{+@653Fc4FnZ zQ|@~1`Y}b%(5dVz9?!u*XK;75iA&l6>zEdXSY#AH<{zu*B!?eMi%p z4TmX0FW;0raFb$p+cRZ1KxmYCUQ>{Qy}B4PA(=955jEH<(P|Xl@R8}|?1B2mbteKu z-b?o7xuls#IOg!p+hRk*2OW}ylYe7u+|szUk+*TzEvx%hwWT8>EfpseX4>VphqvB% zM)XkDwAD;Xc!u+!Q0R469UK10q-AW+Glf>0YV_)v@cr!I{ z|E%ii%*?ze*F<$yb(2C<60MnPWTR`gMZ12g%yZ*5y4upGVQ+JZyBCXVb99z`e&Bp) zP{bu}4kw2!r6=LiN7G!hlyH_8EEFDKB-JqjwKfh58b#q3j*8@+K9+K>;f;vVLgO0c zxhCW0hAKG&!o1`ADmiJp1$FVf;uu?@fiD&URVs{CzdwE=ZeUrifx*??bB-UDlDd2A zd{^8W+@PPGj_zWCnd;&0^s`niuUt5=6Jk%yWp0@tu5#g;5ML!IK4-Z`3zI|dc7>V_#f&p490#OG&C3QKERxTbRGOyFkvA57is%jABohR50?;c~0)r)X%jx(`mr^ z9o$9XiBXr|N?sO`0s4Hp=@Tz-;tqdTc>EREykLiyac9eXXlrwj>+}E8ftOoEHZuQ%3#`8GG7DGm zZt`~l!(T?>-NFbjkvR2xBEsOHs}SmZm`*Cd#TNCdlPsUU zy|iN?5(#F(gP=h=@GxbiL7xeOoLW{$>41PJ4W`L6ph&Zn3&|pY2e{>6m<>AAVH6FW z>7~f)Ai%G?g2bzG5jaqSlqQiVGHsfQAkKJ)5C-!3p@6XS^a!}S$GM{v30ouDh@r?{ z;qD3Uet4E=_0o*%9w7K}-q;_e>`S5_Ka}$9Q@G)i)S@#GgDk75R?|buGRfWD@@qJS zfU=hsRT{Q!3SxP^{wk-P|hq!Cn#qU*G&hoE)ou3E~#XE)QZ(9 zRtRD1kTp2rB3QV22b2SmmY6JJoU*C~icEsM(?E_7AzE^fK8fUscM11ROaq~2w3qs3 zXA5v%a&^>njHpI%J==kw17JebNL~BKLJ?KwBNzsa#1R%k-mpyGj*v#J!TUdL-D?^K zaZyVc>L(@%vguo^B>EEObUeV+f5~P_m8hBI4ZIfs%Ds3DG@=H^wwR3jk{%*}8WNDkh;-AfnQX zAAybh(&x_vzUB)F20|!hAP*fU%z@&3&3@g<>#m5h3c}J4;b~B55F|k{=n-&=2Orjv zg>okRe9Gmc#YBuD@_fKd*S_eFpkA zd09uZ;OQlk*r=UVu0MbdXCOwhX))#vOAL+>0x1-#jlI_r^i5ay+6=NfRl46NgprMA z{x#T~@Y5Lk$Z`1wcIkKY&oqM$+4%d=G>);)#*Fr|#d~278_VwqX*YNp1giRUvxzB> z;rW|I5(`4!r()hKF8ZW(A}fj-O1e_yZK8eBOx;2V+hb`_!&MnR<7na=0t z(##h=f=}$-j*h&5gPhEEc6N5JG>)r-TsL+ftvqIYGwo0eehXrj)`;eilipo`4dy9Z z+?i1b9pQ^x6?EYMX4?UO>nDT)MUT$jWc*4sJH`%8=NBOrd+PfnTSS)QxYnw`ld-S??{6C-ky!{f5RVw42~_ zq|h52ltY+I2oBQF80lmg>Nl!r-%*iJ6*^m|McpJR&|E^)#XdG*$AvER;P&+PT=nqx zZ1ga$ldpq>icOL)#I{JsQXrN1s>*1~E{a>e(WShK5um1_!=>YdT%}h%u;5E6nUYp+jqX;%AVUJDD(%ddeM$Ko ztanN8tV*a#Zc2z%ElQC}t<^I331(<#pk{_jnP&Peu`K24LM_ED6D*;p8B4E9;$|3T zwDt+E7WUQlNA@|Vq1nvwjM>Q8&)Kl74yU<_&gA0qElO~U2d3yJmU7*dFHjNlQYJLn z7;OY>=#DVHN`@Dk6!gvx&eF~;ew`_qsr*`Ul$(*wFW)TlQ0!WdRA^k%Jv%U?Rr*bT zr)jhZHHbW!{FWRmvY6Pxz30=f*e35L&tGqU@i)ym3+_!^fn7OXCLXZw|C;K%vboB; zD%!Uj7c5xeRJO{WlRECNiG7e<<`6=(Ml?b&#j$4DWyWDL!iGdVZ@bMJ`~ijtHPEpCJ|gECA0tK5csxcp7> zGHYHk%kbM__0+nQt2F$yXx1zXJu~m0pRGHq`fD_89Db~~79^PDZh;L+44J7nsauzk zmGP7{&#GGKAJELQ&uZ9&a3G`+q+M|&Su4HQ_(}hh_Gh?jaYkB3hi$9w9Ca6Uv{IK6 zd!Frl**xnJ{L#T7G1o3v051kF8&3;?jR23y<@tn}gpZT6%dpGRzMEOZcJeq&ud()z z7S|e=bsjZtU6(>0XznowTSxCLqn)mg3#*P(3EMoQE`C;N2O69oIji5dzOQU<{FM5s z?g23fi#)74*_`mfO*#2gj@>z0cunb##k-EBA6* zYqQs`fQ3MsXOow_CwCir8}ifeqnr27$IQ1j$qU1}!%5Fk&qgnz&nH@$ciVLLt}$$kJLGJrM>OLbqLN0KM?l0L_G8B0qE&iQiqu-NJoDMBXr}%ByB#(;}`}%r#e8=>}trrmXNXyu0V;dF&(Xojj!W zsbrg|5yPGMdG>fwRq1K>bU+Dop@kf}!a=evuh+z7k+`sU`$&hGy;}Y|Z?BFZvL~`9ybpN~SI@VQT6HyS z7qMblzh_6ca_!P=|LG`a=Mk_@n5a3%gSQ2tE@+^NOyXeXLE^YHsfik0Ik?7a5JMH20!jy24Vy0oH z1^KE9<%Hp|&JN9oPoP zd)kNnKr7U_riwoOc6ahCi5vOl5wm(nyVfRMw}*?2#*eTchmK2D%3AE~+qaR^1h1W& zU5LF032MtvR&iZ)@dq_e-?XoHeA%B zs3@rTBywM~T{be8XlJ-`jI@o26vkx~b*KU2k2%s0vU=qTQak`MzIqq&R?qu64pYqA zG=cOqtLia=n$8wvy8Gp3SK#)e6S!~d`?WLp{jvfu6mdwBQSj{1(QcRBa!Ote8FfK)J!pFl_vW}+!=DlZR03mn6MK!DRJE0DQl`Vt~(Uoj>2;UxGlOfnR9A$1NM| zUuiJBZ18`_5I`gV0wSy;DlH9ss~9_)nAkd*+d2DcXpRFXVC|)}oIpUZ-n@Q5rIjg8 zf&0%{sA@WE%FFQ>+u1N0ey}q#VRW;xf4vU~pBoQwXk+4RNaAK=ZR^D2#!vRU1P^fh zdYXxh5@gpmvoue5O3pY166Z2c9w{IDM5)4l6w$6rb47N_>e{S;cK4KlG#zMrNk}+#9IM_j;D+orRl; zwU(HL4KQaw8v-mWEF65lEBt>h{ZEtsRaNtUs=7so8`aqfpAv{L|RN()eZD81M(3={G|^X9-ZDAl2_?a{av-cB=MD9PIj*E zFWApvy3|l%2IQ{c)CLC7lwo)Ce!m)D?hr=ba<;buuZ-4==CRXzhxaNcDa|K4rfsZq zv@ycWF(9yi-@f2rU#Zy$-kE%%5CZ%24g(2<;syIp0ah>!ywUNuhG-<9f8KD2*ME?Sm3wAffLzFoUljlgAK zz^q^KZpLA^R0UdcU2>4D9%5|XRbesefjL|CM-JM#H^}n3MtyJFSRjDu4~>kaC~)@y zD2l|riBi4naj>W2aKBd+uA%S64&0S0(=B&*zB*3;vybRg`R&TfGd9qHZ=*c2wRS6d zSz*hqpTh0hpE!V`NxH5n$yODOmnHMY(fmCtJ}-noQGDA5VL8F4%l*rpEFVV0{Hzq4 z?+Y@?-p_YC>s8%IQ}x|*kXV{RI&N!$Yj>LiapLXXPhC!4Rl{$GLj1s?_f^ii;F&_4 zM!BhgCdw+?nz-(F^R7y8IBnA#*2Cz4qT>a!*bW__(7P%QbGjcx(`=h)fTFyIRsQof z^}jAl8jhO|dc1Cay>`6ydQsN;+X%-!;OOiXtXKCD!H-=DX9;-qNZ{M#EB2wmevuinZ06trXRa|zCy{WO8>EaW57 zUITuQexHD}o3&fyD)b^II0O1b__U0)d$N1Ov-;$J6H9pd-1#5{?PXy(uSRp7~8M7to(U=K#n^O*D4 zga$>`4~Y)52n}uwUYvai1NrN~52+7JkfaZ5yg7z6;1K;O1EK!wFUSyw zmCKotZ5TBIVQ`4zKT75;Nwy@H&QA-tNCdoa>3>?!4p3X3RxF80e-#8D8K7*GjH`?P zs;*Uh(38L*yACbAe`?R~5SHs=qQ9+JF7}^N>EHxZV_O~0knCS2M1%@#*~=gW4{)mJ z#`Cp|M z4g1dAZOM(E=l@>7jfw425iw1W;n3kz)h^I~BJ^8er}QN0d((J79C5O|JYIxrnWpq^ zECQEf5dB&XYgx_4t(loI(MbFxNPP>OnJ-uZ~%n2q!6`=zl-vU=SAf0epzLY^OQUlmbN)5 z%TMcX1q$J)(j->E1Zkn0cmTbMh0yc9GXYvL6I-^LDvV6FYrE@#!nY>j_~onudKN40 zwqH`zBaW?`|0NaJti54$71TT~`xG2Y_TBVfD!C)I1EKam-u(I)ezW4!o==a0|LM4~ zNYkcnMyBE-0AgR*S0s;UI)w_BjTDrDO}ozWO`?wDKum|e&r?AmJD>%zT@XKS)J`d& zqY&2xZA1&|;jRJ0)4x3Wso}Lz$v#5AQG|&nj1qEC(X6oK^YT>6WdK!(gL@BXEGmK9 zRd5%tpEmVspUH>bYQ)Xr+qIC|wqB`EROs?fG=Ds?%r(8YRvL`8DBzPBP8q+w7!$+{ zDsM}1>xmbC1BwBbXaXPqo%>o zm^C*rr6|b2*m5dT-A@)C;xxw3S!w-#QzhQ9mRd7|Y15=A12(dyMeATmVK@^)iiFfp zMg)97$U5D9JB3ypXOeq6ncQ%nf!M!e{?ke8Ok_Ayo2FNp?VYMuWj-q2DSPKGf`qT@ zPaDqaZX{kY^>tvuu}&4WEK3TeVU#vSqnNOIJFp7~@NRC;HhO(Q!6Eb>ZcdmKxVJF8 z=;r2H4%q@FDn!8$sItmEY2}QvyzfgI01es8d(uMAyfxY8 z^{M&?o)wH-u7&4AgMCM`55aW6`vLHq@|8=tC zFyCg3s7#^jm(7xk9|wS-R6bHJdq3EKQjGw^l%%ijK{I+ds>gO3VNbd%W)A%E8J>}v z$n#WKjgxDHv1vDVUGU{DdUrfWRGJ_{U5Kl54LuKQOMP4}lTYjNU`{d8L&s%C&1hE3 zj6r$_kRdE&zEfW}KqwRP=+EDqrkG6chVY~DTTD{0%lq%)!lVTgduz6w57JqtnjSaq zsOO0nXFc1{$?cIK-R6FM_ai}FTY4)R6-VwWc@5Z1YPq|4(E`d_V$EF|&}0J$6+?#G zTl+YyXq3ryMYHVS>-MqTJykT#t_zV`1>= zylddTYS=gfLU=IAJma8kr0`5F_FMs#k&mfX6*QTjj=7(bxz2jf#rvWNXWS?GTsDEZ zS4{WVFIj`b#?fqDEO0rPwS+Tk+)A8^4i*8IxB?bRb*xJ>P0$sxx(b-|Acr6XTrwh` zM~7;_I);fN^RuT2(TvB9wK%w`(ns;O@ZHA<_1}-=lK*RNafEviCUw zNs;d?0F9mb5k+T1QEYB0|j2g_;b}OO-s3A)Vurl>g?wxe| z01*Fij?442rB^~v1RQaXqMuXCR}kve%7B3G`Q!+o zyRLZMNRdo7Kq=D~o8^NW4zLzwc{epN7KzDJY=|+QsNUqSh>)}!2H$L5`Zq<5kv8^; zx#<%3Pk^}S-v`9^Ox=`?pu6vm@$(8cYDkbg(M)@B%jAB8amLM8vQvWM=FiDYTG{X3 zaj@?}l`KAR^vK=xm_)ZYRrTZ!8z&1RkcP?c)VMimK508`BM?@a+x)U16crRO=K!+p z$R_=(9hhgCCO(g|dBc)M-vZ#W_L!#G6dqGOA(yj|LA+o^l$)!Ydfw_A zuys0<&5*;MVo!`VnG%#^b@fZkRp?@ZdokIHfwz$+7j|8uNhmYdeSnff&WXUm>>9*` zz}{lMaCXqKr4WUKOUO8?^b2nYcB>H8b6rdsw?_=w145)*aUuA~2IZ6Xm*+v)aZf0u z&&)l{C;RuM-Bws8%`}1y2HP#MdkQMtQyB;^FGGPPrI0!DuE$f*9&GJjaFI0So9cA7hq5Fxk8G+kh(1rzpd6|S1b039t|4B6m| z9B+8OKbS#Vp(s-zp=Z_wm(<|W$>l?#)l~XCSL|z2DM{!Op9~n88A}uma#)WnWUh4) zgCCG`b%wMf*g3HhT=}8sedX_NK2SNMZC2_3+n|0jquGmh=ZD|M7(pqEVB=M?(-m#k zn6=KcUA|1?Uf%6w)f+(xFY>*oxIBS8!hqf3r4j1tt7SD;#oe6tE9G355{ze#BCt`K z+nSd4H5-I%B$MQa-g*C&j)6)lyH|vYd$nMer8d@2`f}Rl+7&;f(QUQum;P4SVUs%0 zoqPN$$`LBzQ&2INW22RHdILK4R>%<35x>RedoFKEhw~m*C5NfRJmnU^fNNx^TEZ?< zfui*X2))T6^#TWLTvEU#=I%n>rpcfO$5F%4X*^_Uu_cmvi4fx9Nlp5!9Uzg)wM$a1!yvasT$`e&FaUi4B+-wpbOc>Q+Q&Yx!ZfBiAczdEkH@_8UF5isPGe)zAbgh`kJ|p`G+knv zQ6u%Sq4Zap#Z!;~q``$UjY^^W-uRvwmqa>knFt?kaAZ)wmMVIzBSYwrw@XP#rEl^$ z+>j7RnYX;j4B0e*?b5$1RM7z&D8UYxvC3X32>40ISG|_drZAThEn$oAhha$;7Dx(; zMlDG)Dk<96X{rIZ6Ag>F==fZFUwf;;x3tVM-o!v91{sYK;ajaG$y9#<8{()I0hcv) zDejd7x5yt*c_1Va7z`dMTfG30)QETZS~shF<-YtbnX76)W|kA|Y$DS> zA}|{9yV1$)*R)-dqyaJmi?`|~*)8*!7-?~9H#VzN=YZc(Y&CC#@jj2dfi_sC1@>iV zNI}J%&2YykK{}fV;tLF7ysRz7o2pAxUU#Dxh}LjQ1PX5oIa{~_Z&8S`E@70PT6V2h zvql*%vz-OZ)3j_c@9NyikO-L1(%zupFh9a*(uTb?Du6*~M5=>9%m<~?J|uzb47?}v z#r{qo+}~jr7Y}bh!^w-TQYZ&Y+<6fK?Qa*Pjlj2Vl!a(b7iI7g6iKl6u6;Y4VZK@n z7qo>WeN!sregxY^>3OGyfG672qhQ>;aMn~8z_3cmd{?$Qg48uLas#E*VDsG?7Wu0E zu)jb2Nc@H4FrJ4)Q&WKh`5DSxxJM{^53!S0A|5$g&yh;yYQv7YEF6S|^rPh0elT(Z z6geD3$4)v6ZXIT7qts=1MDF^+hA14C+z?MK%D>^Exh0tUf`yq1mHaP{a=yRuUyLsh zsCGvE-+``1EvzumOb8&L52P=Iz?lK1)|SatErY{*d3H|wa)UdpF*H8 zuOSDwKb9|P_`*axo6(`C ze;pS_kguHxITP>EFm5d~gT9Gy?RYYulih^O55$PuZTA=1NPJ8r=*yIQUtD61+|dWM z*khp-+s1S80)3m5yg}>)efGUvolQ`(IRG;jC z&Q+tkqdlqr@-mo#y!-2ITLiUAx1@r_J?$^Jy}d`MOB-AtZ4ZX26Hc*Z`FDExf>m}y zx28`c=QsFh`aZgPBWASGANGoNl(@arj(hkDX9c1vx8tHDYc8iQiqpMO?}PV0?mwwF zKAk>GkC6*_CD?Y|+)UrdnzUKm_puQTOez9rpZ48*F61BXSR#Md3sDpd>x<+s@)pe> zANMM`Oc2S74;~i*C&yDFJ*tl_t{t|jm=q_fn5>}GjqiSR4Z3J}C%-t8AI_&+pex@* zEy=!BL3w03$pD8;JT%M!;WznCo|i*8Ydb{3La=Hq9X9f$?LlFSeJ9;%(H-kv2R^c% zlnuc!95lo-cA)m@TU6 zvHfH@q(j5+)t>6Cr2fPp*%wVG=x)xV;-ZyzG42IkS7G)dHB}TBkA2c-IY}99@kA`y zc6zF24r~b;Hkk((E*GPF^HqPNee5S6FP0DKjyvG@{k*pj%ECm&+-B~%WH>wL$}gk| zQz~tU=UP`m{&v9%l_K|+qqEIga-#UMPceE2B)Cs7gMNGINs4$U6e; z70Ktw9re<_9P4F``d+qYC8@!ff~8L}K9)qVlXjQSEHKm$PPC^kv*Aq*i8jB+-aCJ@ye~a&nfjhAK8 zbDKE_U-k1QKh6kJv$0+60vKa>I2N8x;RnApn%{a&hyqmqDCkCo3Yy3*+cIW|shzLJ zFwA3O4oO8dlOFrE3>yzn)gd#i*CzKdaW{m-9&nFP~l`d4wi>f?9zU~P+$0F?I)^%d3m zcI?yUnq1D!%>P?0`h9@#tiuP)2OMG<@!g_fx#em6Z#7Cn1yn%*x3(#68Fq}ckn6@fHDhpA=KEcQ=3xACMT_t-QLT3>%3A1S2UYe8A2Afa<9)W@0 zs79lngpY*B!*!u(GOy*2J_po00IpLI0B?Gg^t2$;E4F@AX#N($FQ?OCLEr(kFy|5S&LSJ3;3}yyexPXP2 z)aFGfl(jAzfQdk{24IQmYn;lhf6gslx%H}=dx;{ciU-9^@AT`l^6&q zEeG?zXi`(L(e@7IR>=APnLSsqy7|aOC&et0e~N{je|sg!7G~>{aH=JN!MiR6@(CkkjMrD_(-nU972F8 zg`Z^h?bibUNcB`1cJ~0_Ba^`6sc7};XZSDx71X5%kgY&bKgTB^oG8_Kbt27wez#d0 zoR@-ZS_{JO`Dxewwe&LU{6X!~BtWsQ1;obpV8Jr6*rH zt_R50&;ki1hBu5ygEoK@xb6o{^e(BgaT{-<_0z`()XX>`@z;L#1`rAY{yvU<bh_xI)$DPO8DO6IsFP!{X$^jSVV4xUU%5;GtP+!}xyr>QX`!cJPx zRb=2zGywUU9yD3{ST&B6f;e?^imh_^FpfJQ<8^5v+G@1y{_=E#4M2Taw+|=nYQIdA z4TJ3S`^(E00Bs@nmaig6NV{1{o`^BCVBB7qT{zW(H-drfi-Mhyc~2+;#zJPIg;-8b znrS%0_jfDI@9QiJmdp(y4~ce762h$730Xcfrm` zt;3;gj9rIN8%tOFnBWUv%`SjrOWqCx;0*DdTphM#BpzmABLJ^y;I}f4jlCVBH%#ed zw2&I)w(w}2Y?EW4F!eFH_N=| zrw5G;?)BX<;`}Yz{5t%KL6ur(@d-H9lXF0#q4m8qL&x3Do|4{{!A-smggQTeL&5Hw zCmD?-hyvtT(Jd(Se4efh+MllB69K%)0B8_-6VA_|vh^A+)9ZQxjbQSHY#zvafbSXL z71Cr7&#>#X1L*$rg(KUW7RsAPC@hkr>EjB;S)P9hr z2*Rqe)s^Jw9T;~FS~-vv?F^;L!~wuY&h5PYc`Q5tmbC+toV*Z@a8d~tK;9AlM-<^t zGCk!{Q?X5CqI;%NzA^qA9XEZXZ>bYAEQ>bHN8jN$;W9xY<^c3nh$jWg@v^x0Ta#|g z3IJ+@bMK?|qlOqPyYHw~+O?cXBx7fW(vAV!jYxpJNUDo(yl^r5@cU2OwJRCu@8dBF z-JJkdTp~84<1|vRDD$kR^1)}?q8-@qpo5>iX$VvOG1WCm=K-Pc2a++8g39L}fP@6P zGysZ6=6LPH`R2fyS2LrdiJY4v4L8_A2E~k0gyUX(Q#=^1X5E5z>Y^I z0=)h&J7J!=KULe+$vNa={ccg{D@3%L*Gl=sWs^j@En{m5e_-Kt|3Io&R`3a{#nUwh z3?pk!PE7YbXj@LIVOIcb=P>ISfk2+1+bPdR1S=Gf`?S4nNUsR@ROsX0FV36(=EyE` zo33+*Rt}^W*0?j-f$;n8;B$w$u;#Akkp_-w60|rUfCp>K*fMb%A2!E z7(|MyS#7Jpy%o2O$Tt@2Vha2LH=k~fbsb-TWCZ-lf=e9!6{PWx2pV_23IVvRlgWZ` zG(8@oo+RfbRU;Yw=Icc(+4T*GAmx>$(e5=b7(y>yg}IUg6k4X%OXX9*l?WJHfG!-L zw=#|RK~u7@)b z>2*U7peyp_tgog00?=AqYe3@Gp2V@V#B{q|p5&)6@b3NlJ)fR#R@6LZwxgw(Q)L{kSL_W@p1QkjZaDC zTi)Y_yr@#WGeo`ns(>4oFTTF9)ET3I0m6Pw>Ag85)nUmEK&9bL76G>~U_&Hl!--8c zj^gfeE&{Uts{k#6E;K5ZEy7^8g>%t74}=VBE7PJdrR(6k(2$JzuICki;njj( zLP?O+Bnvns*yb_!OTNpADuH4(*wygi8btis^*-5O@ThjG>l46wFmywPPVi*2 z&haZr3?}lRzFi9@{+#9W%q>SJ|D239Zjz{w^k%PP+uI~i6 z%4&x5r1ZVI0X(87d7l*4ezCi8se}_ac?dk=rsK>hQBNuo4(6nTTuxr)o`zgKKft5l zN%dd{^0F{dJS28|Qh0?nc&7GU#gPi1#6N1NB+497ll8!V6cMnJnz1-%FU;2ry;3A8=47I_2!HW;8T@ zD6O#S%NrI+X4$`VfFmPJ9nDqh<3Q%onerR@t_?X2G1Nm~1M(h9qUpYsFk>DUWetZ| z8%!wIjj)1FIJ}1#k4>)W%3Q?6yqG7dF0uhA;VJ3F7LjaAq9i>Jivj)?LFe{n0D_i1 z#=N5l^_4o0DC3zm;?zl?k#54klcwf&D@;T}XTN_p#1_nN0v4+z>w#piCik`kw5NVw zMUkKVakd;iXvOz=JU}IXX|1}KIOx|EYoW0@y9jt5k`k%SGhpeG937{(77hFn!OAUf zH{%p@YhWu=M1b)8D=`Ro1|XEF-VBjq;H1B@YSw_kkBy_5k7NLsLnaefN;mDsw@N}3!Y zjJFHf^xNhMcP&hPrtRa0XJH2pH|NFr46fwS=LtjpB6$S3eREbv%u3`a7Cpl8{r!_C z-7G(t-G72~X&R_OKM&`a-I|Dbp%8iO=Bx!5-mNJgyKVNz8ZK+WL~5T0e-u<#O*ZLe z<{=|JuDed@FTq=(EpkWGV&WNoWX11<9!=27FB zHIb%eBhhhA-6h9i=wOR&3wj*!-NW@7(+M>anEGZmc8ZJRK?uOo5h1&^@;RVpGG0$# zWEm$05?XS-u2@&giZXTJ5#{uA%6aUb-~_2)IOWtNB0J(S+Ony(AOb7$?!wG2;Oc<* zZ_r{#G{+EqV6SBsSo~7ISeTFi3ITjKN}2KxTNknDKbY~~o15qPLmRACos0Nzy=EW{ z`Mi7+K<>T2XgJ;S!sjT-J{N=UM;Cmv->)5twoBWV0p-(2&hhgcnIX{B16WBPgSP4< zg@zN5T{Ch=uQ1*z=lGo`)A_D0S>c&Q7$k!R>Fro4C?$_2ubhuyVp{|T4OsPrpqi?$ zv3OEBH!|16C5?|mcEBJO_R3Ki=@P-HA7STZ-E6Uk&iAW)@O=+NUV4YE*m$>;A%ZI+ zScHY?^W!SSIA58?p`#qWrgw~=c^ZeF|A0kY>T?c#KJ8w{ybfLgGNstded(N!)a%c~ zx4&0R^2ymZ)^CG+->QPg54FRah-X^Xygx%rbQH%XwjC2KGfjYkno8n>bDE(gO$EO_ z@{|>U`-%KwK#<$?PRROmHy>wDA+dBd4>iviW+Zg_%{5$zo~CZ@W(SZ8%xu4@|MYo7DjIn(oe zfH|~Q1YDt2ld){bRJqM*J%YC`lqB807$hTTb({dIk1hvmSO@)m?#Fb%avFGZ+Wxz* zKaAXq8nuCV#+4>CQ{9{d^)MrcTiD(Z4zfANsJ1(u zs9T0~D%*HR`P);qC7esI3>H2Ati^gAFl@&Zmo93OMTn!2Yj#N7DBK#-)h^o)Dm6y= z>vHD1?mvxGS>0)LTydtby9xF>s-G|1Qz^n6fl<393yc$2-Ab6^ad}Efdi|Uo+rT6z zy-u9df{m=bGp4(wmgM`kBuU^P2K&7{ASgw0n&rG2ywJJUTr|Dd0*(+mPqOUmckOvo zCHGZM!C}{Ob#qT3GF}_R&X2Yk+5ns6L~PKdJjAnv$DE>;WUynX--p=pE&jSmk046S zo&ZkzAxSYSO(oPK07~Ffb-H>H_Ei{PZJaHE%Cf9PN5L;C_0P_VfCKP{u{5>DKnwtB zDs2*1t#PX69sP!(WzX+=rau!hZ$|v$t=9PLjnoD)e~wEI>Dmu?Y0hOM#AihAo2Ye3 zkO47_ta^q4XYx~erkaX?;!6GW!C>{U$?IP$h^&iu(}>C zwHWk5e5?4=E_mB>5z4@Dr7 z3`x`qUy%n|B?B%sP#y7|{l<_jrs*hYB!@6K8)>Ec_pnGMJ7|BDT!S{IRd}@b^GgS$ zOE`X!tTVB!-SPT|BKAG}MG;%5NHRDAmy|H`DnxAMnKtb*-1Mo`vsqwoEjg*#_NR%x zj+kA#dab2NRKOfxZx00;OR8}U(I>!P>|FQ zVL85cBGo8|Y3!2fU)e-)Oo+XMiI&c`hcM?{PYV#~UhhUS2K@*5(t|EWYVv*lU(X%~ z1T8sBZ0SUMA+47Y_#_chW+I>bNkYZ{Yd+F^b8AFJ&!j6BZrNn=J;dasR(BltmDn&y42f6jUmkiedhQmPH1^*{8*N6g) z^qu+UB)_)lx>c}+7P=mgGzOaK_XlLI`%}&v7fJGm`edkDcQ%Z0vi(#|6W%pB(hKzw=|UAV(6O zWnB0ZZU9!MHVRM(-8^C8T$2b34G_E5U%9tJov(88mkzsJu5}{_v^PtTO!Nm|fBOg2 zW(y3Oi3Y}~&h94jJ5d;e4y->^po9Wp&*+aiB;z^%onFdQ0JaQ;MW+K{1)Nj(l>Ho=By zdpPE0F(1zklV)n%{9**?Hng@UxEd{%ZxzL|08woCCm$u~$v+9d0($l;;VNj%kteTE z=QaRtG8X_uQ#G##AHZJ{LxC^?kTvwyPd8XcKkJdR&h|iDUP$~C_?E}h)m$(8oK9Yt z+i53sIKZ;4CF(WF1QwYESYo5q{`db$KxCju(Xy?wEmrCass+6!NR3`O1APDvknXlu zkQ6^fUGZ}b0PQfkgON3o>H%;UjW`-e{)Pg)r1!7P_*{S_XLLAU4fga5#9+$gRI*I1 zwetX^^-3>R{c;E(155(Xx5m|rmSrV^Ty91l&~vYd@Ne4c*ZO7c0z-+%R{}GDP!=!( zTz^x5E}sZvSHuvmuo|Z*fEq3M+;Dr^jSQ*DP&+OHrlNFE{w?Y?FzkILeC|)?h(Z$v zR#L=p13T9(|NlL(=i3cr2Y{47ryoh^Ixk$_{(N7`hb(ZLjRpfxU**fS9zw1iKY>Bb z|LGb72-$RQhheGGNr146$;20U=s}B}9ew%n#-I}t8v_W*i_eiKZb+~w6*KA8^}Sp$ zcYb|bnW^uZoBjGe9N?ttz>;{OifjPDvP#jXqq?Nm#~|9?OeR9YwmhB>8BG@}f#nV& zW$?Htn>=kL>5JVQ|4O*6%39-w{UGk^N}id|n&W531M`ZX{AU|YX7>WI1*Lo{MwHwk zFy&x*h%j0J?*Ux68bI>H06#*#gK87N`8CMjG}K>=@`eM;jvh>Fp5uoeVPIV|Kxq_+ zss1j=d%Y*r@&hOs6gko20Kyn0DxO1_a|$Es|*F9n{K zAo!x{Cn6fRT1NLb^8_Gog60xY1bd}68TkiR!46>lp*-XNSr|TYuP94T=rUP_L<&TO9YrEJ ziW{J(%1)NEF!0`(yRG~FF|YDD zhAuq%yb)@MnONz*G;Hd+E^2->D1-)^0Hsp)4A`lq6Cg|7gBVV*)>tClmv`gt8&`#C zw#s>UnhHPb?N&|`vGAx*Y@3g;^yZ+)_%0$Ej=ukl$KiRpoa)AWD=ooR$}l8Nl|jM$ zb-K~XYzsSgC|jQ;BexeRFsQIgL*q5=hJjb%Ar|NiXcx@V-k*@<25^xLxnBYL_HXJj z>nNA;fH4rU=(;Y5zoL+D_SQcDIR82TrlA5jdpn-^1hRP!C^6z;uwgL35`V7)gxLFz zz%xAEN41>SF|?Zx%wRK=sA8mXu@?g44%l>V=LwQz~u~96iqZIS@q86Bn6n}%y3a~w}h#wH_PIE+qOWz&>0I*SlnmP@QA}c&Vpm=2; zhxGy6Ow}66kRTveB7Mq^{7=-(#`*B(#EIHmaj@x=3wgm30-zj&+QA=*PGu|Ry$QRi$V22pm-@xcIj zR~5ke-d1&M3c2kE2myYvE)z5|Q8~xKB0x9n1)k@@NQ8V)UjhK%eFA)|1wqy8ANY15 zsN@pG4_h3E=W6EJ%D1r`he@EpYSx;j@C@biohLmc_A{Yf9WMlE*}p=9WpfPA41=qE zPp4sD^)RleoRF$PdkFA!le$fO-~m8MLIo0EQD~&|kScamQD1vm;2Z%}UAC3w_FXcX z$mnh>sV@i4gHs_Z@CN`g(7w|9)rhBw)4~l+kklIt`jj695E9$T-kDP)JdTmRD!c{& zZ~RUxp$U-D*Ja;TH6JoVTE##yC;(A=2IFh4pbnhU^>hi4a1pY3Z+u>Nvelx^>J_N# zm5z8Yj|X=!eSnOF<+p0!Apy~jByd0v7_Ryt{U4^B z@Q*3aeE-T$mwDwfhZJRc$l$6Reh$rwkn|-6dt{@OOU`Z^*uZ<_s>C_~V7&WZ`D2=`qktQr z=Kuc?_uk=Lzwi5ac#^$W_THPadD@%ok!&(jkwn>hXJEu-iQ2{Jp&V?i z2zmxPL!k8O3f_YrP36H}isWGuJ{I27lho?kI6EJ*UsaqJoaPFRVh?fSJiKpZ(*AG+ zdW|*OKK8M{3B?CUcWED{pt(*KbhbqBZY*SDl@ECxlSA52n>r1E%2ilgX52@&jxOSDe{WrJb3aVd{v=0Jak6FwB z#I{WD{74m^pe^d3XEfV6PqJLBTwK-_W<~E0eU_yg9?`E-F-F-im~q>=u@q?#0yBS^ zdBeb7q@`4Z!!8lL7T3wB$X>^}@u{wM%(EWdlXrN-NP(KpMX5lcr2Ny}a!i6@d`ALqC^H#Bx2 zvm6XvJ;^LN*#DiZ-+d>Ob&{`2lbz(NV@m6nA(ir#_tQT~@<>iLABEq=YC@0$QB%!9 zW&@dQs5S0!apAqnBq+Z1xT~=m?;(;*p2AyHg6wtf`4Fv$U5NV(H2XdezsX*jgE1pK zMZtEQ1eeF1L4N@A%FNZ*(jJY}OuTkAJ=|2DN!qvpUOi6@{$o=p52EP6~zNF-c zc&7)tL_$K%i{$mY?=D17_(Lk6Oms4~=QeC_@WBXF^1sH_Q<_bboF@HvW*^A_ZfmF0+_715No<5P=ULjpFRZme&3 zmqo0Od>JxD8u0;XDDP!Tr96n_If0DCL9hLDK@sf2g7 zCulxqEvIB53Ep{N2RX^wB%McfVT7*V!b~_wZ4E5We!VWHuE<7%W{pKM5k4;!<1CU9f|~na7$B&znOH zA(Om*f^|ek$Z_5W*6Eg@>c25o7W2KJPdL#zQbdu6&^cbqII5cxzA5mQQPIs7Qxkx` zv;)vm41<`wtvI}U;(iP!8*^)qy?2AM;Lfq44YIT1ED5*N?Tz=)m`5?7q0$N9G+LLE z9(3mQ)8i^a{K{yweZAfXw2PM+1^tK^4j)On5 zj=lR{vsDX|>F?$aO*s8^kLdc!j!`boO~|-oUIhM%>!H4&8;f=OjJuTpDI+QQ9!|t3 zwfcUZuzZ;huEmJ}|Fe%h*8a2m_0t8-=<{dS*zT$E%fI=>ao+DZp^J!WFy*#ric*}x zd>$PLGTaFcgyc|8ML)8VdzXT--z^o5A7uxHCw_kL(LS|TNn5n`g@fRTpEdzoYBxNy zoip)Df;FHu)Q(Rp-WMok!qAM73muLvMsCM6@U@ zbsDOA8$U+p3m_ac&8w}&hhGxfN&+>f!LYf6t z$T?Pw%c2T4N2y;hPBXA1$^Uq7g4+DZ1B8cyJ<#fnlfQ#K+F*kxPOx*XJ@%gRR>KZl z-aShp4`lb;hst&QJPdMka(o)HV-I^BCJgp&%uYDoK&p6bUzQ%?BBL0d0uo-ah@%hg zx_=n_7gG+rUZkQ6k6%4U#*0hhjno3+xF{eDd&)OSpIw6t`WchlRUyp1K(a$(yr#n- z$iubGqD*w=p;{-uwjBT;Z3b3(5(x&z!c}~nKoSK8zqd0GRg}MSR0=)u{Ah+~+GoYl zNk%IgUFK2|^O6>IG|^L9CW3(%AsX>FA7diuT*@3V4)>2&5E$-qj-D zy}ba797IjX2x2%Pi;36xpZ@F1dqW6LVe+PQ)FQwz&I`aN+KqN2Bwq*-KHk59RL@T5 z@896^i3|!8%j%**ViCdc zlnPS}{#9h?5?e{|@FZ_~`Zum4>D)}?6>os+o)&hPoX|!2$Y9^){&&m!OX|#oK@woPN$=Zp z>B1Cb{5_uZnF+{~#AyPK@WM1N(Nzw3QhcA!zO!I|fx>>)Pi$O=WAAmG1I_+3C_rt$ z`mXv{0R#>YVqM{2OrSmG9#D+}B~1(2k_Qz39PZPENdv%XqIdkIBj{yI`U!d|&$bCr zcsvFcP7DORI2PQ_RAG?Y&UaH7P+n~D!F#xcBwT(cLwRFO2R1L6IftgRhgf!< z1<_~^19(#pa0?<>SCiWvGoFWVXkK9panu2rbLFCCD=P+yzZ;)IAnGoP7vg)P@N0b~ z+7INPESZo^g)6f+S$_HiU)xjm&h(39$4Yp8*$oavd|nTMFdnq!8Us=b34F3svas#< zfCi2YaxI}PpyI^=3Zd}52%(D1-w!7yN02L8`CC5AtKv%Jt~e)?DzmE(h0eWeq95sS zTmKyWav0Tbu$+ei zmhxfic3hU6|04t}?S@DpeCy{|L265G5iK@0shp(sf-!w1CYv9T1~agadi2ml&*9N|DHe=+`&m;LK>UdT{t&+j9M z5Q5B;LVEQG#1%C6+}LBciU^LQ0b)mRo+dF{BL}_-x%VP>0`LMLC3wB3BA}ZkqI70h zn-d}TL{JOXTzEpA6}As3iD_f)#b@|?w{^j$U>-6x^S9Y=%KA>v)js=5&R^4*D$~3D zB%DOSqCJKk?S%GYvEzU_t|fz*-Xt^er{jqe0MzDeI1|9)#Nnh%{s!YK`1ScuoBA;$ z9Xwn?aXWlBFwdZ{2n=t8QF1}b(sB(XT`A2RSS_BgxveMv2Ogv9l1o9LZzz>4?&kF~ zq7D%j5Ff^+|FC1+1z44DD1%vo5GNGiMV5AEEWEMY5QZDLi@4}I$SAq^UIXo}HXr%2 zv6{m;5vv6(>EGlD4ANrBQ5QeaD+KsySs`@O79kBHM5ar3h)yCd|MmOVPZEXwMT*@4 zs42+S0U%myVQ|JP$--Nhr*$iw@`LQd97|fPaN77-rDY#mFi%{hGz5EOZ&J_^nw+N}JZ-Tu{diNY2RNhG4 zNcFvq5C_AT77K^ui~1}gRaXxJ>k>kU$~~$}h_6aUsQhx4qm|r4B(*GFw8%s7*seU48 z65Y8e&w6WJ%W9)W%Y+^Jp26qGCBY%TriY-Tb>fArcL%Y7Syk7y_s${F+(w#;R5KXg zSyL20Spuc%Gg0+8>rObWoMsrK4nuK&DWuLs-*l#dr(EX$#&*H-31GW$EsJ#}w?M-dR zazjzpNUeX;XF?i1_AcdXAM-q_=+3Y^X`<;avX*gn!$4L37z-%~UA;?smnu&Yk~{My z7Q9RS0A#9zdeL(Ke7X+%t4azfXE2%wp9c`!D2V3T*L3Eod7y)mj?Bgbz#f?!w*iou zJ4teyzKqt+`iBNJuekzN4MFY9(asJZK&wp@&H!l;9WV9t6tP7l7PtBD2Ar~czBwVr z`~ixK5^;zOevFSvDncw7o&R^&2Z4a(!EtMEOa)V9+M5HdX-iNK-Z?zQlcjpOmF{e9 zTUDlv3oq}`c(-_{scq+Zgdzd&0 zIgYM=KJjLuuqvk^KmGyMF{sdi0VdpRKxP#oxhOl7`qPXZIwDRB)7Ds^xP66 zBO%W97TB1qA;aV`)Qlo=r|RvKKx{*h8xI;nOLyt2AhR}gJfe#@>-qpWmaS>9+tyE= zjU#v+XDU$Mu;XUvb@PKWjTh|Z>ZM)r9LWeO1IqfY>^o71=kYf`U)a1+J1F@^;3_Z1 z^G*~$=g{c}!p*_tew7SGm6;cTI;lzDhoyi}W|HT9L{0D6n9_2GN=SymtD9_%ZeBf- zhnYy4vv|JmxisdU#~7W@z1cQaY|WZiq6&B>9RHT|7ym*gOk^QjkqzvbsjuaEj4~Li zlwN+kT;>(c9`FJ{UNn5i@lF0AOxRZZ!zuZ&fswOqv~ zYJM2i67@(`+Tryu$&Hx-66d?c0&53Q?NTq>GxhzJ&>l$}JzH5jQq~n9AeUJz$R{r? zMrpHvaRQia+PLOH5{_{cKmf67MyjFJ7|#x`V&^69rl*2?kJ{P1MJ~(NPhaajw5k^B zcw`pPHG`zlsp&6>)?|K6w`&uL_g%yqmv64R6tYwzV@DM;<)`zh!90+Ks5!Tyh5P0y z!83*E-#gl5xBdBl{6t&Dco1-|*Ruwq$AZ{yW!Fqq^AFhS-(4xHFmI-Jd#a8760*yj ze?c9I{{4q?quV!X1r5{(_~|h>i<5H=D?SMz)ZbyHqgx)NSET&abz$D8dR_yt-Tulb z$*$ZsyJwA}B$7F#OFs{h|0LhO8KL#s5Rdh4bLB;?`V48`Gl7uRkIw*$yK)48x3lIS z?IH<%=`%W2baN!M`t;gRUkkin^D1Alp$=$_2W25$+CrO=v@dLx`RzuS8@QWs07!%rTOi|NKFvHB#h56L zlmavJvUJdt*UQK%le$!_x9ok1%_g_r$1z}3>6uwNQ!P13JN1!1PLwvwnf<9gz*1(M zE|Sd7x+WGuk|)k8!=hbRiKF8($kKL$Z|+pIMGPnEvRaaq1z-F||Hd__UaKBFM7WaV z>`(77m!+{wt^L$b7Dg~gBtB&QgDX}%K`ka_W42+1F)MUwDqFX&;H#hAyubw^GdSKb z1d;B|J~0PD#zS?j`4O>i_8i#1uGlr=uv1wSvPrTU_uJPCG+CpANa?*VnVj`D`YIjB z$nOXO3Hi_z)@tog76+ElR%C$EKoFXLYmGOf;mC#3Her4WBvSep>NtYPLpolj6QSZ6 z7tI#ji-m8u^aLe+=9oqL$bS`no`F|-pM!KOO%#Vq1G56x4)WrF&N5{oFVi3j+E?wu zeKtPBYOgP07W0E3fEsa7sg$FQMa8NAoBQ6X{@b~~e+5>@-D|%JmugvsQ`<8kqj1$^8bAQ_ocTmL9ls;;IYR zrK%m3JhReE)i{%N8gCKl&qIv?J}27%4nh(8e^Qh{k{Pl<(+QwUj)BglbSO@^ir2)? zoWPrW|KRp{N>!K3I_02M%u9%-jz9{AoCKua3I~Aw|EAtL+uVbF7CI?P-sjA_=C;*aSIo6R}svNAcDv? z=Mg)iH)|49;IP&U{pwP>>YLxD^~SGN3*bqRg9KgOUGH5aN*fPJs-(j+&7v>Ci#jHe9th40bMIz&cpuB_kBeTeZ`{=icv768A)^$g;0|peKW;LOJSss5!ohW3M1hG8X6_OXkqfg4N5|K5q1tIIM``l+k{oy4i8{IG7HjsG!R20db30w zP5goMl2dA(02exc3Wf&T6-6D2VjzR}lBGbv51IM5PVgEW;DqUmMeb^i3luCNsJcNkXJ{3aK@+&l0%-ychR=we2KMW^9YF8feT7Q{3& zyUc;KwQO;B57qYf?JpvLQ1J zsO3Wuqz=)GNHgUO?S*A9Y_aibGC4#G*7nA6JIP&n0r?`D;2w{bUX5wq*F<(Ocq8{4 znUSEvg&Q7|vd~fR8)6Z!N?>ak0Yg^Jmx=Y7?Wf^}3XSCbnwUiT@S7dqn=@Nasb z4#A6pG8_g$bTae-3kD4eguGA>*UXQAB$Fl}3tkLO443o{84%FCuYbI40u8!55pid5 z0wY#-l`P#jR8BfF+vnt}^f0xM=_>5L1P?F||Jgk}Cr|F`jofS+!y=HpFC;{~&}A&#LuxP4B;C(AF(CnguJNk`pQsc1g9 zWJR4QIi4SQ1$M=|IIwEnuScfa_tzueL1YG+KNdWO-Udu8kE(bbDaULCOdUnFF2HQ} z>EN%&hvpkNW@(UROe8sVKjs_MV?imG++ZOo!;e|<0d?*=USrpBJfBkAM{3^x&-aup z3g1{lsF*alD+l~?;#f{K0p3_Nyhh+tadYy`Ft``ZV7D@Y-bzu>A-#}Fzb98d;S+^K zLdf8OjiOkNewr0y=6Gj3HuDR->Y7=AibK|GUrpDe6tw5bBtL#k;h2V2p@1d3vh2L{ zUzS!J8h#hIrd=#KgdU#a%2P2@2%3B_?j{)VcTz(TKVLZy5$d}rx)gXc_9nhBs+h)z z%?B`jD%^|sJEH)D-Gn_LD%i7^JPmDhyu{wz*W!Qu8Hp~)j`7s9D>VhsR`A$#v$Em*}*0i2^6X1qC7Fa7G#JC31K@Yi(r z$tO_<~$FY-1d3_=6JrrFw^3YF&zPwvU zK7!|`>yG}v8z49iqcUH?r^o@GASroN1su@Vfj!}!gMcue1Pk(XmrcxKQAeD!&{jU{ zo?iJ{=-GOwI`Y!+$wrkCy>v)JWBz?c(M#j&TgE|GZhQt7y z#ca4NFnR>(Q*OMn@8 z%o4hc+_cs6ZB8263dvp5AN((S;Gh29RI)Tfpk z*Z6C!KCzq|M=te2LB7V)PW+IfNt^MvC$Bf6e;V9k+FGshp?J|& z_QIC$yWXf`>6T_@5c03)Z`t#&PQJb4cJg-t2ZnN2)}?ppi%#5ij>A{w44dV1_rg%J z#PZ}HA2m&^md~26vXPj(nWqW8whxl+^0zKW{9)AcPb_4LbG*&de9x{@M@HXqcA*bL zyOozR(pMP#k|{c7S@Q%I;DUcn)R0JoAP!3O ztE!2-S$c4!!dF%nK)=|qMs#|H_?}JV&UoH4ftV!Q+SaFv8}uqmJ7otf=?~lP+4iv) z9pR_f-f3li6XFGm-kezTm z8@(C4-+m^OPtSRBsN|PlC0w!{^dX~R75Xh9S6V8w@uwWzTxhTp_@2D!&~LM1@(s9G z8+7)Yx$zbGiw?aLZaEL1egprTDDi^_%l7XWUm6Bf&RGxsqTe?){#wEAI;y}-Iz!xY zI9}qhJ+!sqv+Mu(ch|Bt?{D|ljJpZ~cZ1HZ=E(Ux-}Nv#s6B{&2-zi)a;^0K*f1p^ zn#ttd@6BF9`jp_U44}wN>c0%+lNR3p#_^^0`46j%>p%rw_=mkmN*#@q=J@(2xP$Mx zqLhh8+5a}`PKs%kGbhLfP`lPN0H>>>``at+i#=(Jl@U8!T7W+OmyYWvFhIZwscL(b zSE66F)fLFcDycIwgC@*z4O>F{NYS8D$l#uWUIhB1b9mCi;Cu+2J} zG2*Z-B+o@(s4x$X8%|`p_+!~edv}HQ*lxv}H96x`--EgveIVlq%U6qTzBPp_8gM9K zODIktFIW^PdTxEDr#pj8tSs)b0FD_2r$ z9(SeFzv7V;Gt_cD@ATB|Qt7cA!IG?e{ECxllKbeZf66H{f~Jx0k9}!%@4nYB@Nxj3 z%%^VFAB~caDHxSmJvKj>?j+WNkQam-#8R=dzzX~tJG;dfNu8?2vt--TDaM>CJ4-L zg!x4VoXZAasG0VB{<4cE$J&KPX-v@XgIEA1Z4ZG~nWDA)`Gba zjdYQr(KK{a`S;!i9~o~GuFXKV(+(gNzUCKTt%0+B!C&j}*1h%S&-wNrN={Xpi)wGz zn3RI!yR?8fEqpuVieF2NWP5v-&el$#Kl;(7S}|2e$Qyp$WuvaQQB-OWohRE z8QTnP^$L-$C$={}>mj0j4+FeR9%*E%gtPu;3zP0#EX6a9Nah`d%W=C;Qf9LcMs2py8KE9woMy$01>2uyk5& zFcfPM^_uVQ`@G$1@v8i@CG|b?2V#b`E~lEtrGT22r;r^ zIJ3#~BMxHdB64lEeHO-%s6UoO^ZME%n57ecL)0pvJN`VmN5FHWR){wwm1X(FsuEQ$ z^9JlKI*=}w!q6SyNS<>Nkw-F)H|*#=cfG@-0%)>quMZj+q<-N#w48)FICF}Y(Owsztnj_3Egh- z0mE2EM95b`$`)+Yb!tIbwlgRN_up{Rd4rh#^lkhZpMfpF#+#&*RdfXWPjfB#++B`JVGc6-roN%or}J6wt~}*481Vs@4j}!+gF{EtP@fUu zdu4VVGDp*yXQL&`SU`u3)_a>Tue0V4?T={n2EaGhR9=`rw84l7O9IyF_hV)!ElDrk z#Ln+=vkQ8Onhanm5W2s?ctq}P5Gle$yjh)eS|#=o!nRE2SMB_A@PbzY&NiT&7MAoh zN;T}pJ3PBNS2X%k69V6X~*Ah>SW+T38gWPkT zVOYdcEXer-VTiN~Bozbs!!q4uek{7 zmM!O=!@eutN*roes~K{=bgF^9^$&1Fqqe{gKYz1jN|+s?1WLq6X?I9{DThF`%`Lutn7Va6frm8Wx2 zA`0qCFAmnKM2;W|F=er{FOP-wi}m0oT1^f!JgzsNZ4U-VN8L}ND5bNVg*lfRoO1u# zjTpL?6lN;rS;-{C8xqOt;nGIAf$jyPi^%@|9}%@MHqTo2wNhtP1|h-@GEzU(dDlzMwGAXOC&2HaF#R_jq! zVw_rcmD%w)Prn@-u`g0*cF$g}E7N5WbJu!{QQN`|?tq@sHn-JF}!x z_&w%!PY}Hfk#TpHm#&Jdn0R7~G8pLXkHWo*3_w&%Q4Zu>S_l< z{gm7i?rs{_O|-t+(6?j!gaE*EhD!XymLFfJ6f493v&U`x4e$)!2;c0EGaz z=#nLnzq*BxGUuAFk}yf}J7mN#>PGATzC85>R#I_l@&(w^(c0 z&{#T=eRfvHIevk$NR63D_L|l8$}c3tQ3}07sc7>0l(IfFp2uT)L5~j><~$PBKL(1r z(kbX~tzi{Ym9Dgl)@D%9%H*4xvX2Im1-FE^I zeu7BZ`5CnQDI)aduorJW{w|Opbf`E#*)cWIZJK?4RSNt|WTVi3Xni%*DR!mn@LVf!>uSj^`HgdJ7$u*6R%{d6M)BJuZj=burUa*yxYkQVQc>Dy= zoLVE!C=Bcdel%=cRQlosTtnOid+a#I*LXyucUax`9rA|S-OpS!OBiaW)xb`0NxwcI zVSZH=k5NFUrxAZUbn6cgtU3j(>2APzb?M)k5}9{7Jt}=h zC*aI$iWqYGUD7i=_GZ4t0iM5KEaQ_y^>y(x}qubX&%em+_{=V zBKcu+G)^h=K*)^pa4-Fv{4_2-v8z=Jkl>nc)YyORj=d*$LvCNElG6*csMSBRhJGSK z=1m1N%9Sr7weRu92!Hh2mv(WXBAV~2gP>Xa$B&`a_6pO_p)%PzYJ=nSdyJIVErsR_ zz{bsf3OSX8yTZmpbZt|9$XVv=(*^C3o>#2V4CK?_-q$-WHe#r!ASYO)K0RK-BTei- z9EOXjH+gRXt4?PvoJu!`!tBf{Sz~InA36Du16F^{%S3t_o43D?tv^2;n_{Wkfc#rB zE}qzUV+99WecMcrO~Ot@_mawOROb{wR@Ete`CXHxz5p9cL zS3y*?A1Af^#2|q@Ym{`kL8{l=Dsa#`DfUr&xnqgajHIoor~TKv_5?(X%wc^AmzY%; zltNUo1rpDi>%I396{L8~Y9Xl3*dP4ZZgjtp*ri=^1gaCoCYwD^sk%pyB)BAoAo@Q`@Q|K87*z zp3>Rhp?8L{tDQo{AISG$l+!MsD|03#!Epe#|JuhE{nPBUzJ4IS3C{o~9zzHE!>mbM z2G8le^h1_6i7Z zH(@cbi+VfZv+AC$e*Dg=GH~6wKnTfW?ytLjB-@Ci(zdwgl;mLF&0hoBB~pr=M%ve0@19Mqhf-4%3^~wl9N3nRtdkTFkJ% zWF*vBvrBl<$kNcWP040Ew<1g3aNhe^g+T!x2Mld*&ja>V1Cqw?_ zN86+^+|3Lw`(tB;$1Z{F7?)W zH!;hio4FDjjzCWH|96nnqKhowXOLWp{4c5DfBEu%fXn{> z4__XHs1GNfT>t#Wzwx?@KPWN9u-YS#(LhZ3|CK-1RA%7s)BzJ8jcZAXU`(!jh(u@* z^2!q#?*BPo9^r;8O##OLU$lIPt43(pKYY*Q2HgLX?=p~vYEQpCK?Ap0;pu^}P!`N z2j>!=KUCTN3N~cCpflvJfA8?;?Q}L;%kZN8-lK2!DQcvYhNus!=ctH6%UzRCDG{DF zj8ILuuWVsTO?039oS}q!*ld(5ZkT9n7{PojE+$`$0GrV1C_U0Hq1=q0Gmg8fwfU>7 z_AQlPZv6R{m;LpY>2Bsjq3glZ&%TX+*tem7M^N-D?uF0F5Noxb^E1uaM%?#nl40AU zFR;V&-;LsT=rz&l;ttj0VY85K1msG%v4IWbovFZ`6~o^%e=-FxbbB%IS~Zpv(u$yw z*wlCST%Z!Ojo%#hhA~ucOTfx|RIz`VA8%17B#2#<4WEdFnJ(cpE=eFOsh|WH;c~gX ztG2Y32lJ3rIO~p}w#K^oQ39=~avY5?(@jQk_|Qg<|1$W${AF^&5W45f#KXHlHq`%k z4G4W&;N}qEMnVAiuT&uB4t?y0eJ%-?fvm!_Bykcse zjq&Ag?IfS`oLu>7T4e?tvVbC`;=Z4y6x{lKFtx=Wpn~?)O?X*t*|0cPVeAb7QL878 z%K23)C+4%6hD&#n^2Q$8+9DVnr%q-9?+;*--_6Toa2qa)JYs357?thgztB`YR(tUR z)SM;Gsxt@*_!=A`Yb*g9$r!6L7xq5z`8ZEq7x>G!L+(4cS72AWk5_VL#kIgWJWJn# z|2$itX|w%h^yJFLpVUFA=+}6E=LRqFevzfIrzp%#rF5NKFdhfLL=v;2kmJjwaf?I% ztzJVhUrThM(KQ7>0$#=#4lSjdc<&mz{cPs9KHa&?w9PJHgWA-@JCLBD?QZ z!HcnKOJhKOs<&9bO2tUZ*RkP;mJR9})GT%)^K(ZSU5{Vk5?PxkH-*fAdfLYcr$6Kw zs0WJAA#1Jr!V~@Jz@HZ>D`Fw$b{MoFpZZjA8NA;kTh~Tw_`+;15cQ4{F-7H}#grYP z86dM1G8bH}edgTdEJ|J@)!Ui)g@zSIoE>8>``b(GjV$=RiS*nS`}0(UwL?5+)mi8r zRfEv`r*y~BwDvMCAmtQwUY0B}9BLAgdCrx+8PtOlmmEib9KoRVDE~$Mu{K^bI*pgxlspt&eVI4bT zR*51Nw53g2=UN zX5LCYSo-KL!W-G%8NdyjMJv58NL$a>RifNL+?o0@F7`4t9q4}2z^+nv9}cXOb1Bgf z9~8zXx{Fc_XR)iA!F_G;i@KPsyKA?Y7$%m4T3}QW6WV;w>}h;QBG`K#8D5r8S|yM? z*X3z=j(|wOIS?3?RM=)UEHAopfg_~qx6X}mkl5|RD{5 zPhm|j#%;W;pX}0m??m8$RV7$gc=u*Edv>4%J~>V3tuAuJL<0pT8lLba-ncGwH~yDa z;auwMB~sLiB)RbqJy+cVg^M1&%kt+q=@Rq|`lp+?*d^+|aQc1?ciaG-Hdr$Vjo7bW z5WV@;TE4f}YMLg)^L?EXIA~kS7feK5Y{Dnt{enZhnsk z=V&_xW`vzbaU*vKmoMk+c#~sQM@bG=|M$DFHn0RkvO>z+gUFEf;g$5rL-1Sz)seAD zgdL6UOfE}@KGj!-((2~P5C$&6Q5#R{)40iPIPd&ESN8{}l(G(Q0stV>7^k8kbivlwj0ZY*=d1Mj3;+E3u`xQ> z%V$!~60?N<&s}gLqwc7ul)GLa;xa7o6AD13!s6NQj3jV|^}VKBPrzk!=1znZs@qNA zLX++v?mKebUUu+$A!ohjRiASiN8%jB>vbYt`O?>BPZ2-<^PZY9Y#sQ&BdyRYQKOT* zQa!UDk2*xYMxd>A(WUZ?nuTi>d9-q<;MujHuG*;ODL-}}cd&`bEWDWXg*t*B({A|K zLqiYFzI>>XM>}2IsfrmyRyTeX=rYb^`{^E49gCxa`COq1h=oS$dBGM^Wc zIU|lY_9Wz&O}_?;S}(E%v4&ET!~IZ8Wq|}ZdUN80jXW?w=QXPnP$RmhjMXed6e zViFFivU*%;m%ad)JhdfgbMvV`9(V6nx&N+oE{U)>7rn=^AKXPMmfN>aveZ=elb^zI zLhF5I;afi8(Mb0WjfrtGq9`YpX?GZ!tBQUnN1Broo@{|4Kgi7EYj7wg$Xrg^vnb`` z{h2JrzuVlQs-yd7g!IXklt^K$EQBS%Zt20>Cje6`$$d^Aw};h2c#7b}41UoPNw)fb@ao>>-NW`lXcsWcFIQL2b^}g z%0IHDZru7aJQg^twwO%^iT4koW`21D!(brkm*tG|_V#vyLkS)^y(0Hw;2!yPYj^Dx z$jTmoKruxw2=1aOcrr-t8!W3M5GP7x`v}C7+L5D=Ap_1}*GSR?V$Tv#uffbWTiFCSl^1Ed_3qwdWD6BR#tkT|uWrE=B<36*yK#k>mw=ty?jzp~t5q;^N6Z7X z(|HPOI-3#5{ z;^BEdvufJWBIuy$#8V6U$T)A!_ZY!(?39+>JhUiaJY6Rmmh*%x}|75=;k=rY|m!$#NPEN@Wv z`|pny9F_Q!o_TqP`7Uh9S}-Qg16iUYWd;RVUw}o+_x#pwDrkO9z6bwkD0l@D)ScTf zDbDaXBNfJFS+_wfW6udI^KF_a*_y5)>c##t{b@yA;9H|y-sW%Q0jBi^T50JfH>m|I z?oUX%4z-K&TQIQWGKl}4N1WK+z*s!|oDbdItVhUc!Ji>;UPt^>K7)kxT7rm|e!kpl z78%*|(?7F0pQPN@zbCUa1r||OW>8t&gKTiE8FExv^>o*uh2BKQ?=NM&N!pUpS$IQN zIf7)eo;k;{p~GXtIv$!)yl?W%9j&$$r^0ZFk*Iiq6c~)pKe!5iz4g-?J_NSU1AUQL zhNncuZ+@N9eDMo=Vtxt;2TIF&nwHg0>0G?Mv%(6kpHz?#gNrm>#>4~B{oZDRptT(5 zM=^ie$D(rwG$Zt*?Iuy9^qN@*E6@Z_7q+!QvP9?p%p1ZJZ^7xnG@+05GSd&}@whXY z_1ZOWw1;9j?`_VBPAr_d9cuT0&_R~;VTUTF{eQO$VoaXwg6o{8lumhVL33_3pAOWN zmq+@o$PV2~#ht!QKR0S{(Q5{O#FD^YKO9Qcyal@w2StpzN1CsHmU0D4E=uvhVx23J zSEhJKd7)L>fD;18+jQ|6&0lfA!Lv~RH=U<#JW1yp(nHR)Q;PSna=lX)dOC}FQ92CW zA69Zt_U3D_)NzKdM_)ZxWTi?tf0{M z?dhjW#p_g?po|jpcqiaC*;tWz8*=~ERI_UJRalLkEi58+GpEhc?6gmBMEKdWVSGeZ z6_3P-lRs2bIoIkZNy$X2l$_vD3qBkcF;be8O4Kk0Ybc{8DY<_#$Qcl9jY28w9goV;rsp_bX3gBortTH{kNLD82{8X0Otdk4$2@z{ z`4O-mM#CI_JinG;cMWmKthRYBQ=1S;K(ppf+90g4$Y9oBXBwMb>ZKc?!J&8%e+DmB z!AF;+eA@BcsrXknNfInZzN^PPwLIhqJ^#;cPeJGLuG(wU@xr5VyUPm$KT4BM{iqEo zmX!69Sq|ii}J9pFMGw{@mpuDt@T+M>wkyAz=7|JRk0Cw|=2Z z8(}kYnvoidALX-voWv(SU%XI$Y<`p<>92lxq(+($_cUo`f=s}wQL_$<*O&CHvZ|v( zLSHhX4(@*rhBHp@>+Dlx_99W8x=1V?E&L~zij1L1@5kQE=E!BCtu(F5+}s9-N!&2G zttRvIZ(O*zQh!`KX7j^mJ8}+#|NNQ>qiJTH!@df;%7kQ}19pNWD+}evG)H0LhO}~_ zK**g7@fp_#VYxMtLW*-(0-WQMx?sQ^6I={wS`Vv1Y~+>ewS zfuEp7PF7LJdkt$r_|~m8vX1Slp?rm9#PiNa17j?YfX?$;nTuM{%pZtmJ1o?3Ii+mtI zWC>mTi&?=nW#RLe`!vgX@_6WzH^w*<^MnqD=pzZ!C9Y5;${rl-vZ=*6xt#vMPhOV{ z0|^Eedc7_%zIZfML?P|3(zus^9-d-uV^wEfW!)$ zx3+!4$d!`z%+)>zWTKcA{+%c{)@$qUMR`ptO-jDPEuF2d;wnFx*s;FHtF{7urmd2T2()vCMvJ@fvpDz zbKp)2`p|B12^zr};Ao`%a_-)w0FrAw5AU}{f?v(58mryHtAz|Xi@L54LReQJWOvY#7AQmRTdw__vh!i zj=%}q;O7A_X_G&+Gx+ejamXjJ=8OQa>f1e3##ExXKhqcebiZp>M+|n_y?VhwAh{3+ zWofeLq6V17UM5hl+#$oocBlJ#mi^SNZ;&{z(sFMwC<#PZvMhe-!CZF`Z9>ZA^QzHhTrFg^oz znsPlhq8D5O?|IZV_lDwr0B~@TDL|B+Tpv=a#uqfU0ixc-Y~8wPt_b0T@zu??EJI59 z&s1~vI3*SVrr{OK+Lv;ouEJV#(bv z69(Te4P?!^^aK>iO*v+sdb#*L1Qo`C6p`AldjKYl9ZUtdMBxWa9x6@TMi6hznSOfB zc45rO*kSZkcy`dx?3+i<{rK~&1L?6i_H6IPpQNu>QlM_s+B~-!h>7ggV_C)wEkrVQ z7!d+Y%@n5F>!R5Tfi2dMpocwpHtkfZQv#rBPU1G{_s|IE=P|!mK0SM4Qe$QMy(l<) zn5^clZh`B7=8kj_cj*29?0hjx+k87MqH+_N&}Q_W@kWpLmiJc5zi-_Qk-p*hn=~mu%3cl<7k5{t_PU+<-Q2gVQ>Rtj*2UCv)NYUd zr2~)HV_}hezIK-5;-x3jacClP2twEzPT{bLFF88=;H?gP7IiC+B%BFr19N;o^Y&Q! zaAXa)gFFUJ76|m3sQ|vbMNk?0`U?R;PWi9U9M8fx&`twyVM9Zc`*>uojSY`uHNp*G8&8;jOKwzBj8y_HE39UxCK_u#go+mCM}&kx^KN37rM z3*e@AFPVw6g%7lEv5qh%10VQw@Aw9yQFDNyBIfbTLt^KeaA`3(+9d=2MLciV# zvU-p1kZ`MJKqmtKpPWzAj0mX-L&p*rm?D`)Zs0QOrsFmH)nvywKuiF4JNyu~;A~%4#EV`Ti6^5Das%R=uQg=^9`nZeCr? zxJsw4k9YoeljPgoB=t~v{AB$h_c@KO`6j(PZ++J|i&OE_#Pgf&0yrJ`Yp7EzXO#qN zTFClADmqaJkdI{2lAcMBr}-kMe@n*cr)_1E_UP~EO=Y|i@Fw0DzMKjdz)}8lYuvxL z+S!H(;kbNTRqSQE4VXS@vykWMef%Wrl3XX{_}8bCLB1b+x`SAjNV4Mvbr30jO58)9 zw^!`=U~Ug0dT4d`WNwA8G&U@J*Kit_xGj5Otl6GM|1n~X7JJo2?uN{!|LNHfjbmE^ z6rz2{L$Mf!lHIBd3t+M@kS*>z>g*fbXQx8&-C-dJ#K%JflBhz=8z0vUT_hX-YdI3Z zB*HGP3d@zr;+bsKFaB>^qi^l%)}Xk-uKlWocD?#lF*OjH(1|PZ;ro;zTRV;*kFQwz|+495a%i}-yCvgFz}VIQ&UqN5S$5c z**9`7y*7ADRm+G{UuNA;uL0PU-mwSNKOj>pK!L{DZ!R@_c zkiry6D{+SIPm<#wJ-iRn=Ft!huX$E4=tWn)u`S-Nl2;pXTDFeqP{(-;1Pk2t7oh$y z9wZ2K_Aw!GXO?*Qk({*X$F?O6f)! zD^W~(T%_C$h?eu;bAD5$%6r@hWd%H7ej09>XX%g8klF`TybJc5W7lnK1V%F^Tlsy9?HKTUOCQN9w!k01*oAio7l=AfiZPx@-{~$ zB~w`{P<$W9-J}yH^#KbU^Qyp3Qx6{if7!HRG|b0T7K?l_80%g3AWK^bWAdN#`KydrCN&IvoffAnG5q!NvBA&nv z&t4*Hb_MTo4Edz?4EY2?A6yI(>RE@$vBQYxJwu6gqy?((4Zq}X80;``teZ#9wWlT# z0?`lHI1l^$7*vB(S+@Sgk_s$8J$P-{&&Dz5wh#~`S@b8C- z(1uju)+>on(ghMU+YDr6+uILA>4N{!tNoUR-LPoh-X8sa_&rWEEZ;O${IsDJu_B;y zv!crCm{tC_JGf3JKEoJsCx+98)v1!#VoZ6>5`vQNBpbwZto2@%De{^fxvJ))EW6VE z{Lz=iFWFvad&GPFf`0t@bxT&ZC{q~4f!SUO+7|Y@24Y>bQ#Zvm$k491)u8S9?Qqrs zkaz3z)e0fQw4?|;cjo3nPfBuJM?6`WPDE}z zEqb2-S>Cu^QC=6`)D?}Fi+-rv&y5=5v)zR8c)NvETt4&OxNpD^Be^Eeel^-!|X~xxU!prGYk~P*%6MxlVY~L&!H2J?cM- zlhP(5HnUhw!7B;KD-SUqAtRxfgZo6>sd9aBk+S5NSfv&EIZD(eoRpB9KSZms5EpNv zW4b2@-NFRF77(Z*LuqR^q~<6H$+^BddoY7rSCfE8ddzleU9M#CG-1riuA$r9ua#7U4b zO*DsiL_cJjQbU6$;ScxC6U=$gL=4iUVG$4CJ>aEf+=If$3&WSGkWXB}MN03CFMpd( zaPj~4j4t>8bw0`kz^xU*G}_R2>3CQrA)-rYk}nRKZ$~I&sdNTmszXUC9OqhJ^Kp08lwP|5watP-eF|3hv;ejlDv)VG6_)}?sDLD-=ZVYvzLHtxZRaH6U= zs-*MITu`{s|5J$*g_4H7hq2jMBZYwHgWt)8>1geba3$oj!xLE?9_1gwm)#-@qZB7c z6LQC3o=}hMxkrX!N47EAgEGiAmJF@9TNqD|{sN%g-@wKq+QO_WK-DS|QFxJtmz(&Y zc#i}5qn2C27vQ-3yM4vJ|D4>He)_Sl^BW3o>R98*eZDSDE}x_B2h^G#^E;O$`IbT@ zy(yK9yS9AnSw*?(vCLZTyGuPnrx)x{QH`nBZp^C@S-)lU!D8fjl@ z`=e>DODPH$Kd~c zSGLPlGb-X-1M8ClPxDnVbn-WkLO{|nemj_IAgI1`CYsy4epDa1>Gmia+c$@O+*uvm zCM!H)5jr3udhMo;M3A(zRgsE(iX_`v(RtUUX@B(g+~jgu{E?A&Q!kw|x(?qCsO5e) zc=&c*Wv8NqiuD88^?0m6gzI`a`$bNE&ZytHIULNKNQTzQ;-0C7r0knn184NCb0L=m zb5KwxDaHUzAimfzo#Cd&t!NSN30u|&KAeNn%$}#i{F^4 zG+|UPyWWvjnWQn@&K5Gg$$XIMQg-Mi@l8L%7zjBZ=5%i0vQHxP3)q|3IbH4cr zymj-!b_RG(nqAqF86{k`2x?U8+Pao-C9N&+;(J|%`q7W|rvgvsKW11!YuBNPW?f zc^zw%3>BJXs2wpPtVn;#2*f`c?I=s{NYZfDx+zylyDZNhXq|7ePRmSlhRFZzMfb^2 z^WD$9i=FzaX!KUnW(&vprGx!Nc3fkjzvIzVOFgJC{ zb$v=u##1mILSP)^j-mqrv^s2b8#Pl~msTQjy4&hKG|0REd9oF6Wwa+YOG(Q2MXqPllWek%mp8TZusc4faYFp)`yIp@|jBER@^2{{=*fr7M9FCUBwlmi?D6y*&uMEvsQ&%0rWWv+km z8wfLeFa;nJ)7)=h@$~_?N#c=3sr4;DUZmjSm~kRkP_^r^_Oto#&lx#yK(eml1|-7n zS7fes57}MLOZYNL$K8)%=AbFfBdk5ANQ>3gjO+OUw>{jbaGh4>Wazu73r6* zoioP@MDHc#_9114oB_O)_iNHD0^;6WS1+UT7^8U-@bN2KhT%l#mnNFd&Z@3Zx4pGU zH5_b@yPx?&hy_-aXXV^R>TbMup6kbk>FBH`bdZq_KnXht{YR-&(XtaD_2+f`j=TSh zoaU*{r27!_UQ(Ld_WZ?V_Ll#{241d0kkxD*TK860_?9_WoUu%vxr&PsHQiQIAn>*j z{(~F`>ip-(uNU0Jl2> zoaP1l#K!MWKBsS`rd7Lv-uIH_<_c&F3@nT{yZrw4uIlV7PRMRPpDIWP_6ezaH1KMQ zQJ^g#?4TMEPp89+>&@}Jn$CdFSTgH@G?d1MwqNVs6Y&=$-I8M)gPfQ|PLx^`GhADA zOdZ{Y#APg9(=oN{bj;qJLMEq(p(>i=5FB4xZGVjDxB6`)UHq)NPCZ|?b#WudEUC@$ zw)XplI{&>$B%tq28#ZyL|F)9SAZ?k`E|6uUU*+cj!sIR z8o)#5Mr8pMRP{v(pv9k5$|=?5Vpo@bEl-9*1+2pt{dx83K%`HY_c=~2(4RLFca)R)gHDr-+Vz$3TwhEfGwm6B{22ZCB+&!{90BE0{uTW2n$7eEDaos~BuRm9VVjfbnzb4<5+I(G%mV(m6v3M))I0t%Ij=*OD{f_xfI^l=g-d*sxQAD-|IlXnShPC z5wJ9n5j~L36nWD5^DFB*-*RCg->!C^Zw;*v0!Gig1qzQz_LIIbT>%4U1Y9@*iTkXD zW!SK;0-PbmrA5Z3eglR~$iHS~ zYj-r3FvUlIH0p-P>^0|CZ7w%pY}%kY#vG)&T!LC|W(K|*h;_b)1_)_}4~Unn^;juy zj}>9_c#r3WrE-Yp#09N?5w@DH2x}`l)4UPjXD>=zWY!ZFxy&`;Wqd&lPA=D>O4Dif z54xHMuQT_a)y{dwykv8?P*U6C;Or-rXOF)hoiTi~x+R^|-t~zv1UBfvCmjp_fT#Km zElbCx{gPIX#7@NM5XYU051y>Tk9_+IA1ugwY08^d?lS;*^~uua)i~>>QeJk*EgU|% zYI6`uDHkE=hLN^kUq9A*5m}My7}oE**lI5VJoGYA#W*@C_=T&yg;;n8-uT>Z6OT!@ z@3>tL9`+7(PWy3(Hx@E;S<8=H%G4~eIk}!6I?#Jk?$irZcHFV>is}O<-VCL+`uOvC zm1Qr$FJ|2%eDkC;+J|;; ziOJMi8x6g?;SuRo`k*qNs=d@^{ba=dLfL9nJC4((A5X;T2`W(Y%GAi_FL6~LhMKhs zi$Ps6ukD(+Lmgk&8Sy-EOk{gqqX|vsJ$`rfSzz>YktyTsM#TpJnMeanr35yT$174* zj>&HE&+!%}3%}Uq2^)=2jc-L7_5;Ax0V2!u_VqmzfnY}BnB7tZk#lLMR2Zt}{xzNSxhYO<@?|0%cu zb6F3|t#c58K1e)SVSN^AkfF|1?UYA;i)ayDTAm>h>b z=Z1cUMVKlXJITlpxc3^++9OY2?IkB_+BXEzg-o;2pJ*ct-AGl85Bv0VNmp%5KUM9WC%y+Vgmv&6U+pjR z=6P}{Uiyg?c~#Gzf9rz-fY|m|`SXJ!ysT7Wv!f$xVMO&6V}9NoP!-TK4#s=QGX&oWvfY+B-ut-iVs(pN8Q=!qDdr0U*z z0p9Y^d)6FGb+>q4Em+eBiit^@Oy8;e65?BgJqR4JyUhHeUlPX;VU#q$Q>nX+my=VI zVyD9xBlbTTnuu+13!C7@(8V&)>QICnL#%Kzk|}C|E6=wZOSgDM+sf1Zx4*B&1S3so zl&*R%5&*}KQInt8j6j0oTFvh(ZbcsKw+M}5r>LOV@rgR(cO9OHzCPMci_rcI((Yr) z4bj%WxNw}Qv|5sFaQy>m$LJ>`KfHVW$~XS|EGOuM?(;&F7^?w3I~C;ci#01pdMzsnMD6*Wbb!K=-i3)zUp{iivd-rxki}Ei?mhcnJHDE?S8$XQT@(XSxmv z#_+fJv*l9h>E4)P3Z$LV!ehmwyY4gu@C&Nd;brIPI~b7uCE#Uuhm@13S;>EbPUiyU z-<}so96#{aWN^>f;kp}D&+dqP+x6py%i{`K8`T!+g@fZ9!!nFr(B48UmjE`54x4zTuQ7wczA-gD+B zUj!)chq8lJAd^&k{MsPf-+9#7lY5Ap{f04mRtU2JcB7o{!kA3urPqbnEcYfXq=gG) z2HNB`${v116t>zP7vE#UOld*K@4F4%yX84v22McOKi#XBa9sFv0B}wB+6i=7&H=HqrA5S!hnBIZbhFS`TF(sA5I zJ{>hqELUdgFn3-k><4((%cI6{!kz#0^s!2|vX>A5=nRcHr@#8I0^`XABFjrlAR&<> z)$Ra&Rpl}ubO?+(7clqk0*H|D!zSRsraC$SX=e=ES^>i8aQp*=&px1}5RgiQ8{^pQ z4d3gJEFRf{QRX6ie)PrjZholx8MJ)_I5f#=HFxD2*Cw#b?huB--MSBKUHw>6|Y@E@;%psNc4ZPHOjcaAOh>F>L0E@MF5!JR<9h8qY zd7R7ipxo~}MhXxKW#SBI;%5iNIF|2^-`c+J2j0l3n;?KV&dBg?Wa=p zpa$2Gz+p=n0Fqc;X19TZl%lr1;gzA{Cv)xqMOXwM$>bQa>CLVD1bU5hxX#Q4*PCq! zBbLS&EY(;A{9HH8o_or8^-8o1BPUess|=WGXx^?9oh~jZ&4&?AVp(9uE@FgH;wPeqB z<43#-PuB;U7wFK{tEk!KEkcKPU4Wy=+1)<)?+z*?NBi?minxtDyO_JFo+BDOSg>PT z;t1w;!rPED?E{dqpe}{d%@cRgX$?BOav@%7=I=I?12=q?iyL3MT>bs+i|&EE#$Vr| zicyfw{s8#CpCBA&AblCi%dLqy%5d~|R7-%&5dWgX9bhAO`0*Cg&n58`iIh|1<=q<_ zU?E?Pb+7oZ{RtheV@Xc6|1vHsV5!sm{?R*Oj%Jux@O_f#mBxa@0@kLW@>V`r*KyS! zE*(~m&Sw%q;*GxQw5mXx4jz0KOG(Q<7AND#Cs%R*B}5vGVFD($Qt=4y`AU(;SV(dh zK#wUlVZnV5u#@7%WpeW3SUA@sYyc_9YirRo4&>-N=3+tyR?Y#Sl=Bmcsp_dgH+VPc z#np*yh4z^WEVzV{ZRjXDIqYDzD;xvQg&LAmP9AluJ@q!XuTSyYM3&sUgry3z5&uk{ z)AwpQid*l9%lRJo3ZAwgwe91?^DAdCbWR6WtB~FO%u5q4X3pxfl{y!-N;!=N_r>Q} zq0SIP4QDnFa9{grW)%e)i|gRHkA_jFRP>E8<{)IzB`DB$qKTWAnrUakaUl~9$wft+ z4F@~s4Q*yyCc%mrTGG3`-*?c0Y7yaWzMH7mC0ctl!UK9W6~V3ZtC&_k+%P`RXZ1 zixPDAd4!GfjXbdjeBn~2;+xl}&3}WWz-xdzOH2-29G))j8`D`deD7ONn_bq$mHLGk zbK={=ReL47p{SPNZCa+C!;&2`uMc2dlqv8^1S0Fg`T{GnxTtS^BmjYRE+x0VO_Mxr z4DjL%A-B6Xq*<_1~_vU*tRQFKL(Ci z-#9U|K+QH%o;=FX{PMIi!C8-XwHFV2KV?{iMgivKdy7!)1?GtfUHq}U#=|xvu#0+9 z8I3e97bY1maW#K%TQ<(#w`#FeH%d8Q-x$Z@%UlYmIRoCKH;ADhy60^H=0?ra)o$=H zF#BWMpZLLIix^)CDk=^UAHemf@67(iE{*K@Z&- zjPn4|54|7qEAd<^zGZc9?ijM2AE~QVX$!*Jh5TLO3LJx4=iYY=ScH~0QaVAlP+ekN z@zIh>I6(V_>o5niX>24a2F6jNFzJZ>V9ZL?oTLct z4<2Jd+ZzP)h%M3#9o-Ws_8o+m@Y-)-Ja6C7Q1l!>{_`i zoDY%0k0I*O|46WXkz1h7qISz(Nu80NfF~j*n-t_LxVxqsG;c}lVZUMcJp5^@DEU_4 z`1G@l&PNYs*%}Zf5#QuVkf%*eI_F0esb5T;WF#~Q{I?ar`FIfm;5+emoDWJxIP^4| zIt0sBzkB`g?*#K`I7SC?h=~FJ%80v0 z9D>aLHU@ap1_NDP3$XK;E&c$&0Eu=JuIw&gB+dEG`&huyXXm9Bc_TA+e&@m%Jc^lj z`ycIi%)djkW05Aq&LqO^MGSpgI6EMhWoK92g|PF5Hs zlGU#`^yh0=c}^R=hZu}A02D;^>7C^;!^kQK?kjT-k3_-(zpasgVT&fT* zuL|7s&uKu+ABd?j$h(bvtdBszH_lkdVq~&$p0M@^|JDh0YP3HTM0j}(+0i;^{FWB( zgE@~Vn(Y`;@H|tX61#xO@DpM($H1BT0yb9>L_^!__TJ;h)ad`+aBtJj-L1vJ_m?i| zJYN=;g4KtVN4y8DQBSP`Fvw(%^)7z~^Q&-ivV8z3`UZfYHs1>8rGh32z=YM;W0C-b zz!>D3-Ny#WQ6LYK3o(J1)k-K;g_WPFaEM@DM+%TDmseTWdRD;- zJqG*qmoR2yy%cNPKM)<5y4MND*AYNI=BuuJOhOnp??G+h*hnR?qt|NA0dcS*5S-Lp z0N}3x6U4#+$S1+j+1pM@T^Nu0r2Ul=>5-qUrRss6{a@4N;fwT880I)w#j00Z=59 zn(tA<*||^*%gWXv0VG^>*5PcpXWtsVw0LB;CyV*@a+n1x0pUT_^^`aqe`I?w&d<2h z#^;kZ#FI>FsD$yj6UgGa5O`9hU7iUk%#3*Kh)zqpItD+6A_4cw4>BMz8}G`_Xm*Lx zkmD!UDjzJsVcuCqtn&pezNIky0x;^PRhPSEvwgquO+vOKDgG6NiN8kq@n-vKO(LHB za+uU&O97|JjmVCxt1m!9e3~PFX zMN8q}3e04$U9e0HkAYoag zJH8KX|Ib{$dfEQMwM&DVE~WG&?N{V$wHm>SvxgwO>*T~HZb{fHYOYtjI@TEK0#Mq^ zZ5-$NAdH_l5y}?kCA>AR%%S3!%yaEtL)^Jl4~)9Pcstj=)&6z}RvUofS;!%K))bBz z3(vJRu;G(%9EeW_mTpCz!sXAVYpc_UwFFtDL%gavkqIpRkfNCWNT2-dd!(0o}5X zt~Ma0r17@bVi%n%?LMV=aN`W^ZP7O1Tlimoy#`DzJunILAuH3tNNa5R?8-$zj_qG; zc@45%BlWr}_IaRJ+o;ZP7>dRC@|Pn%6H@isAIcqniPxmZKHKAgA44PKy-b`0gFTVb z6M~$F&xjN!g6UPPu4Wh$>}z)f{J*3Uq9wZ=godyLkl6VQoQC)kbr3zxX36={1dN2j z;W1tA4W-bPse6#mLFj$IN3Od0odO{BbmDQ_*rwYN;d@xwf##4}q7Ra}V*X%X-|@vh zqkr!4-Ie0 zRQ4(~WmgNR>DE;}q&yMHKcz5x%du2}{n?khxAlxzUrn(Vl9Mc%gGWUQuG%YyKM79p z^#1Kmo{;w$#f((xoED|C~Voff=nm?kk)08uzC6B7)jNgZ?RA4vT%Ppl%G z=!(o{`orCEe*kPVEx&lz(xiKS;Sm)>i=N)XAoCyhj;k${cG$N=ok)4_$#9%m+5#3; zE`&rZRNGRYM6hv+d#jeTi6 zZLf5B@%lCinY`ElmugDOIItz5vt$Ak>z0t%upjkH#yzq4RKf^rj3` z^`Y6$g1VQ&+da#rxhjoG!g104^e1cFc^6A_R314%75C*+iXam+v0u-R%8XZQDw83* zdmUsAJLG_bEbwb%xE@P|#~ni^Jj4*w`^w~R%b$M+=V1O!@Ww~}h9t2y?*(y;(n5=2 zU@vC=R6^|7Y>K!_=Q0!~Ok^hP#j@hQZG?tsv@Y(u3CWo(+pFK@nGQ#Fz1-+phm_T8 zFnM+DlZk@A)Dh##CUs?EEbYbO);^W;lyLgg=j-VpS+hIiAXvgtz8fqNGzwm6fN08| z?@zy$V!19&hlGuWH%O{~a4E`4RG2hKr2-(-dyMgIt(G&U^sQIX;C+O>)n(+zIXZ{ax(0N_Q9gPA1Q!casfKZ0^{PPv==fJO*bLwKrP`!Gk?UAeGVV^>1Kt~$l^=kZM zh7N=4aN@*V8NKEMRVSV})~-UZ6)z7XGiW_%0Z?2p zBNCkCTwZIXOt%QrwnbeMT{ocTQn&#N-2fqgz{$~q}D4wfg0r}N7wc!|?3%OvvtfpAj`Fe~H9Y*T>eu8Fgf5;vh#M44}FNcu$5j@fS z@p%iYHz0DeV`+j&Uq9@s_aIn(8Ip9JHD61g8IKf*wtDG;BBlv!F4*$b@q{#+EAPUg z;-djlnAjk7p>xH3zwhOUptlgzWjOz#`k?pRVAT5W?}kXl2D<%+lFz?+aLb)X{SfEj zD-T#siyeQE`{~MuBdH#6#=i7qW|~B!#hJl#-o+PbJl(+;opeMm?d1eBq5$G{Q7XL! zpr&JZru@>UL;6vB+L6c*yd_WUQC%YFhg;k-<~0|y2FdWf=Rt?CuA3Of^DqF8xuwi` zC+UVYug`BE@z{8F3IbA~UF5rIjHy^N{m)<{nD&-%8*d61t=(WD>b8A{VkaXIxqJ?B=BrS& zqXe9cUO$*2DIOo&z(NIlLBjrdsOcK-VSgZ7n47n=2F&5rw|{nbgG%C z{+o6zH#HhWVLOA6g@x?9qv<7-m3Q_iiRz%%FtLD;FuX2)JYGQAk(K+x+tL(ujZ1)I zN|zWa_ub%o`3%bTZGOfw3w2z$-$E~ZvfB05jdOX81^YxVe|>Yr7AD-Ep*vwFJ#q7l zhQZ9GFtc2dn9MK}XWNYPN2me6B>fHmF=zjHSlLcTNnZ8~LJZX|D#cJjGJ5G&LE|!a z%OsU@Z6#5+(VmzfLotvr9q?B-$ZMltHl+!$t}SqkIx}qI}^zy>!Kx+g5AINlSNYC=~rV zkLr|lYTqhsH}}u=W_y}2yolRF6B8P}S1ge|E51Y*0lh!nTL0=)t-@%bVvrCro_gvM zmQ^tB7?zGz?0l=?Yr6M)So!WR4JD2PnivO;c{}w##|0-1YK@w12qVo9sRyT0N2EUvoHh5~tb~}lrrkqkc=X!Q>Q}J$s-%USh%co6GUNOW^mLZg%5NwPU96!*mE#d(jIQl?7NSnT4WJS`snTDTE`y3Z zc^;s?<8QweDm>I){%f`Zq6hp6T z6Vs|W=#N`IR_k>8MxjN43W}F_wr)@Lw4>f!&Ooq%C@hkrOl# z5__J4vo`{TfinBb00L*(2EhWAGixG}Y)(bJe@=bo7vTw5Al;Z;1suXaZ`_GT)_}$d z%#@%Xq-NT>G$vRfy5JmfQnksCa;1Mv^^mX0O>i`au0k|_@ZeI?$~bP>P^J9Yf!{BH z-^TpmpSX-N=bJ10#3uVqtNSNtY(_#Xgx@7<`X?dNpEvVq=0fM;OG~A>ME8z^b$%yI zvd=Y=gfCF@#`lGR{dMw3Tc?}*Cs5eXzj31vEH7=9-pOy3y}x}{r$$p?Wu*dCNYLl^ z)DW-AY{<*;{kj&o0?sA1kuTuBzxPCm@+@DN2jx|x^F>FmoiokOK`N~-@MsI58Kdax zr$pIC)-y-Ry^wrFRB7FS2oN3qKlUp-mb#W-HxK`a{}6z%gw^&-hY&!GF}3G?{c10{ zo3g}}$z-wPt4xnL8M)4NTH1Pw{_cz?hNM+_4tuokzhgJ9pIP_55Zq`_{f9R|PGT`F z=i>~J*VWTL#8oOOLD7saz%fl_HdN-I!Laf*;(wzON(@>k&n&628f`S`*cM-X=aiQ4 zUP#rK0roT2-Tmw!O@Eiij*G|wNPfG7218v0?jmr#bhM>_c0xKjU%jcA$a-qNs%gBoK<96$I|(|fqA!mwTO@=tgyCP0%W6LN`cJL>F-dGM_!~jF>3?iZyaP)7^eF#0YlO9h0Ye4Msm&`uo?1n9q)dg7U2@M6fv90 zFvECQ>JS?L7_j`RB50^6w&!L(t&^15RJaOq9~s)NK-be=v zSfD}w@_@GPB~T!|ut2Xl(22T1HpX`MB0yYqmUQkyPplz?KVG?|d9f)nR1ec|jR*fG z_{2JMqX3L62F2=xx&-hy2wOmKSvnWF557|dU*`Hw<|IfrQy!2tIr`k+yBJb;Gm3a1 z*9Ch6ed$d?0Rbm+1QoQkCVbH5sod+fd8pcgq{sA@wa&sz`w?PWs<0=ZG%&nn5H>kk zZm+Eu-CY16R!s;Cc5P4=0+d$-gV5y=f4lM1<$)u6$Rmm1aaDl(Nc8h`RwD@qEeV*S zAxyct!&(FunY#TT4ox5k$M<$n>d0KqP@?)TW+WP=axPR?K_&~|#jNfpK;b3BBsGCX z6UGqInH0D!q-3=37qJ*n7+ejt47j&n%dg#pbuWZ-cv(3tn=+Vq5YYC9W&^2Th0Ze9^DrbC4wEWVvYKlKHexKg^nR@`@LiWU_vWQQPeS)LY| zsbN|WcGr!=R2CE@6^OD9ge%kA<(v)jiLn6T z&knez5xiv`tY`y-b&B;?$e>2OgYoD{Wq%C~_zdHGnGfl=1!&c(h&wEzczz%iA|f3X z1`kg@?kYcDLkBam5B`$+6NLy5x|{JoC*Uh@a)kQ|97(!h!Xneo+=_l1(Iodx||HXhy#tr_>}Y%FFa9ewA=D&-BfNr7GwKFRI!{5eyaX}j zk5s5d;jy|tb{WUW!hl!uZ(DK{&SRj7iSSGV@fr+DTjMXsPQ;%)mW;$C43x47%HWw$ z0vdT*aHKdP*AhOBG2fHrSr|xSoNr*p~s=5Ic$sS%{M*BQcAvl;9S{2d2cM2!Mi+nMl>I9^#!8HnV z&%cj1wMJfFqTbJm!yx0hO9~F{a|ebl?jVeovmc)#E|PhdHER6)=F?z8H+)gc15MNY zxJZQj943{p7dq||Jjio0;u)b~54=#~C(|5^62c4aIL!V^%mwM1Pn*svz;1?nRMyEK zBXl}Qq99UDQy|*O!yFJV3?GKv&5AF;;1JDO1ew-{^3VjM55!W^#6=>zNDCohKZb=F z;V91?soI04M&_N=M>Em?Y83trLMH^oJwaguHsct1GIM%3Csg64pRPCxRO!FOZ<~4N zK_2u3@NfBho`*gqfg}Kgi!iYp>sX&qf*HcIp8KIa`Cpys0dtD+x-DTpp@9~fENh)0 z2II~Ot*i(RFT{_r!^QSPv_pGH_5v_KH=ai;kQ&Y++ZJ%pU#9+?8=|=eNxv}>_&eRh z^h1;++P@!swogX@_RvUWXlMF)ePSgUnOX#*SL=f-rb5v}r@H4RnQG%iSl|zc3A(j- zh={?wEt2eK!bKv(J*Ahxi%H&x>|k)ZFwG?Ba$9+t_S@k^SfTS5?y4AXzM&)~!G?vG!@23gS2$R^i+ zK}a|8jw9kx>;jIkrtIK`2&)$z`_a_;e|NC> z1ycoRiyS;Oe>~Efkedt>E7^0t;5*)w5*{W)#>)vCu@SVyiI0fFiP*qTcRs8*D`T{@ z;gm{;)W-t{(2*}F7F&W|EQ29!mT_mHLb2{p>%$NpIO`v@Hi^)Q95J;Lx}k=VaC*@J zmouA?!9gIou?kNqSvSn&s-l}7r!`!O}h>Hb_zGgbo@ zDg32PTD}H;j2Z028~c=xD$In?NGx!LLvEP-2|{=tI(_dT#&Jh?005!4+xQN`E0FWQ zrSi0?7|tEGlL!P)5c2Nk7FSJ`8E5xQgw6uZSS#5>Y2o!NY{<%Tiuvqj zWrt6Ehoa5q(dnFvS;vMXE&PD2`U}Nyw@fftSBBsV`n6uRQl810%oDJLc1LUHLCBs?ADLZ#9Vs zZ~0B+@MKK{jNR#6dCBEi8@0yZd+*zZxgW(UuL-R$H4q zv~Kmqk0z#<_4UxW7F=EvH4fua^zeOcSRW35(DN4 ztM^yOUHC8S$P170ar>

    qdBK$AP$3$AQwD&O4e7I{9n0Ucga9v2so@Ja`p^b=w-e zK5l-w_|2jN-R9QiS^pQJ^+3i=l1yt4sx6}N-|`gD+?p2nAu{Q;9N94Acxx)>$n(7n z&jbCH_es_dn>W#_g{qi)s-E`FKFHvmvrhX`osC=c^sif@+pbHoG1nZMZ+(BAeC=L9 z^J=luR`=Zzjc3RF3g_xwm+Af?{yOl8qgOBBZTgmbQ?Ccs+un@s6&l4ec-Ln?IyAfL z`a-?wTeC%8o@M%9s%>6xe~0$9uMydE){3?d4`GWo1Q|D4O;S!vseL$TC41FlkRs5vu6vHc!o6t9xqsvAHcR0WlZwEG@HFW>GZ;Bs;G-= zIH|sZX@1nRl(SsW7W8(_)KhXY4=Dt=- zkjeLKH0_{@%**=t-_oUjW~??!k5F1T9tq-P=5%|pcK5*_tZ=5?Q@VtiM_~~e1qokI z^E+8|nZK|uy$=lUw6BaC7s7g*i3$jlKh0&A67F~mnBhG_qcsces{ zyPVLKLy|kN+v(t=&gN|2V&YWM<^OQzWY9Oo1*)j^W~EgZ(bE#@LzQEf%`5zV-#YTb z+E$@Aq*0UX&$)(Z^_VPOg)8S0brmx299cg9fS?|}`QqI29~8HdkD+_Q^%6vMAfj*B zesoC>OGis`vHJ^Kk57cW4h50l$tR=!b8%z3a46HRTYPjEgj{in#3npF_Y&!us9Cm)~A>^64_x?#? zC280^7KId=#mGHHOHQi#1(=l6qJ z@pi9r)%)5z8?Lp>F9y%2rEb;MMQ-nYmi>{2!cnPr?uaECi5#S6J|VjdByD5&-R%5-Gmq7hXybSKJ(w_qUi!s#D~;u% z!LDiU>@ocEAkfu*brg!1#~V8*d*`U~ft!VN9{vVFEs$S))IJI}dVd+7(0*+Sb&1;g z56@3xql(s5_uV#jEQ`Oj`6*Lo*+`uH9DFYP6T(hSDjK~Jlkh{xtO+1!oN>F6FSJ$1 zvM*dGy~C8*NVZe)fl2$vkZHoFSPr6(UL6#>_HBDV145 z#WrN-To3jA{oeDw=bZJfv(`DS{%E!Lex7@Hp8LM;>-r2mE?K8aO6ZucOuJ^rv~e?4 zv!=F0)j_rCI{;dikDj>Nd;sAtZ7$wAs7nz6`QZeR`Q&*5(4p=guzgN2c>$2%gl#|! z^#T6=*m&h7I0sxtNCgm&1N~Y+k_wS_Qkk6wwlpJivck}1guPvPcl|08aLxKy^Tg4+yq+pI*-9;8c zwti9=Q08RA_g>y3R<4KF+p-TOXsHU<~&z>rBEUHo9!OwI`&EZ68n8Mzn)c>UZG=CUkpx`Wb}-bp}+9X1MciD zJWSP_B=p&gX-T+gZ>G=|nrkO+fwcAzSU`^=NdL6M`v{GA8hcRL90HcbmghT)2rwUV z2U$!)$_YCld!Or;1@u%kAph>YLPbC=v0DVB0=3OASWEx`VR1Fg8<1)Q7|hAzHt!lS zaXRvs%pC7mnr|ZZqVaOl90}jx*r}tr2(;A!+2wxYY5;G4MX0+kVLsiFEcFTi9r<$J zXl+sm#AGPE8kV3=J^QsKUFf`SpxIVX!MasS$)d7kC~<@GiJB`=JwaRAqBka4w?<4P-w3k}(A7Z-8g7yP9bXDbgL8mPB63^Uvasp6jXw z0}=(B*$!IEU%0o%3Dt8o8+Sds!4~YtRJ|iS_fOIdV9K^CL7rl`wrf9NyB5Z(!nRbb zHlao}YB7BCNs*k`s}J|!B!!1M{?{iU3CVE>Fc5qb!Evi}n*4_w8r6kYCUmgpYGdqQ z6@NJhN)#qQ2v-~$p~VTXrzf?avpu1IjCQ1D|7Mae8?Wpegvk^O#l@ep%3e8j*{pX_ zkpc6q0ZA^?{G&^92MSq5y}?N8WA>XU%*JJRAvu#(7-4y_kiMvc80JmhuI00n^Q;k2 znl5;H+AFlPc5?|ZkSnXbF_foz>!N%sg;~L$Z)VuJ-$5|rd3tRDz@1~XOX~?J42u62|2P-os1U6;y}*>eDA-j zC`6rH(g1aVg|_Y&H3H4+bsnTMZA)BqLA<-srSkN@~R@??Teyz$xgLg$Th$o;o zhvEX_rx^RI(?vYL&|*H0|1r775x-;*H0>9;_=9c+L9d=pdM0h82sPPP>ED1FwPddd zW&N#LP{+stRH~`Tr&J3ENnU|5_c9xx%i&?c$J%|}^m7;?^A+W!Ww?>;xVB2ovaE9s zAy4imt5aw$7Y{5RPcq>~zry*OaB;VLaJ*WmzyAu8blmpA91p-EKJ)B9?2@%4#TjYv4*H$>z(P3cRu1JS6t(#;Fa-7(D|A{Ps|H<+Wj@`ckvt=tHws`O zO{bU+dB*^5>1NDE(~=ygiSr4_oR09c8CnE3RZ-j{AeG6n=#?+{RKIiwA9cMVPDz-o}5G4nF(3d};Vc`}B=tDaR~`yITsB zFMN*AtJ*&>rlKTgn!)Dw(HBV0X^xK(?V+>&yi7ND6L2`5zOfvtW2%!LX!^1vR$#+|kB_RC9* znKn&Ut|`;^jLX{TLyG9!q4KVIUTKIE!5wbde9^6upVb=Vz9R?RV4zUbuOJq z7_6|HZ6VYHASz^-ufum(;K@P^<9vD?+3Ao7W!!-v#)RQ#UuhwnV1HzN^{f$%Z=gK+ zs0Qb?EvxlPHCqz>tC}MS(I%MS5o>uKrNEBrx<7ITGZv<~Jzv9w$5@Nb$Lb&IPOqOV z+SF8TD$r}_ua>jttdx|z^QO^jMhiJ1(GI zd*wN8m!iyQt*Ue?SWYeUCG6-5^xe{sTK)O->!2Hfqo}fhW?5IyXD5_#PJ(mviw4FqZVYGDl9wqL-)KowG|ah~IaNgn!^hal2GGcQ4OxIR^{!M~0p=N}VqdUu*m zM{iPoGbw2Jd%NAYI{W-X51rSpg%!P-9H$Z!OF*cN2v|8pL%p&XL7#+d@Mg@8Z_*O`-I)5u0wG00MHcmTd|fEkTU&;+|@^ za?CysqSo@S3?FD+Yv*RgrvzvS`e78_pHvq$dlvm;_V5?a(3N(9qh$Q2Lgx+`NUQBI z5O8mvm{(Wr44iCv!r^s?V&_G1mp?ais_*)&LDPQ4ri=Vr8)a<0BY2NtDuR1+xBsvT z-4_=iCtumdsXQ}teH(Xqu5z5FOYq42P1=M=#ir+LIg*~fvuHCCR=ft=V&Ow@K+45) zFrLDmBjMXpFLn*>yWh|vpImEQ% zmo%G7Sb^Yl5Ft3thwG6as{Nu*elEB!H6TKJO3jD=PH1hH){l5Cyeap?XW*Yl$IZVW z7I20{8&7jP@8DDqOFq@V+!S`cs*joY_d`d9aT3*f9HXkmIP5_QyfW&gysC>E*6 z#>R8plbpur);--b?sW6+adUwCMDdNEMdR3rOO*rNU!ERfTkcUg`1s91ML9QD)j%bs zz3fLmd->*cVg32$0OZUGf433bEM$pyxlNCOK=q%6*g*RB`UiP}&SYVIV-ycmAK(!D zcvm|i{lAsqpe7VrUnp~@wsWVpWB~1i3&9yO$dMZD{0Cn8Cx7w(3s9k(Bzo#MU^KVv zS+-a&Faa3DiYS_PFGqCtFQTa}q^dJqd;4zW!bh5i;!Af5@;sYZcbLeidKc zF4i_9Zs6WqB4(A*dk9%$PzcHgNg*%hM4C!!B;dP+sok&!`7D=$b&ykyN2oT$nMT(K zQnW@O5tr#}rbmmSjBxIjPlP_F=VXEL89zc)aU2|cVQV~GbH4z{OowNrF7rjP%ZDO{tKE78- zwIy5$Vk`cL-*dwjAGQ(cQbgZt9P$V65X12tqnhGiV3EfG@S0l$(3S)WXE{J4s{!IA z3(h$)6_5Y`NjJLEuOb&KbR$!L!0{BU#$^GF;-p-GAg;Fn3|2nuwb90rHqsV#XVnxC z&v3AG%L0L#4mm%m?X8tzNeI+ErjxElW+HgGsIC{ng8l%K?SZqE0h~GcuD<}b2U;;y zW_83r#KN~4^s7ESfI$4hS3d(N9PVM%2>=gk??RAF+;b*zOn_SEOUBN9M+o~TZhz5~ z`sxyw`oyvQNM29v_Oi%@aGJOfLW)Rfw|InoV)-_T6>hgXWNK zgKgGCHk0B)I^=zrdmy(VJQbWT%|Nv{)tv_|w-^G$q~Uylo#%ixcY%zQHgS${Sx@Ef zLEFh`U=(QpGVf(*mf36r6Zo6#Qtd2_9tQ(0qP+_ES&=e#ehD=Nr;$%#(TD!5m!KGK z!e%WCiUZ9j_~R2ffXLen`}k%|2<-pS?&ZEJ^ssqcIpqk99b*6lhfD={f?u>#PQW-~ z`Xn=7N4jQ6&MUbZpdlJ5Gb4Z~ujOQ?%ipVDy>OZM4LAFY2(SJ_P@-`8)E~G;4NmHj zZ|B8Fj1A<^L=J93w-JJ4A1KJ3)M3I((o=Y*{70s4KMH8t&9Gewah#l=S1i$fHBta)8wQHn6x7A` zdHRnwXJ3_Y$$5A zed)Ofd@k-dpQQU&xt--Ei|9R0@#O7G;N14~5W>tP$mPMaz{?Y+Y>fnXE_KBFc_upe zk>DPrC+?NQ$*j$YK0%3sIxV|?}Bh1Oy1o;45rL+8y3sJ}tQ~(NY(S$*g~EKIFZ#<@CH5A#~7cu#UdOtrQuSxPOay zvBa2gAJUskaH#KCnm+Nv>5R2QwmTimTw*=utj8PiOV)%?HZOw%`=83h9G*`2`$VyG9Utq2ETytUud=Nb+o+@`x zKlLTUlu>kE+Ffnoor!YDNoON8+k@8(UprO_NrLI0HZ0(AKSzHrezU}j2cN5)k9@r8 zb=Z8neNXC&{9ol2>fZ&upmT6I+c4T=f(!&QMwC7BI$r;(()s1nChMLC!j)c^+&Aoo z+TuuSwbNEUk8DY;#*o-IyZ|ci5!Fy5YlnwxUymB+>W-?coe%k3YuQBk>eG36#c)$j z4n9!rMLG--r^7!XO#~W5Y8*x}e7YG3Q{YDX>&pvA$4}17k)`E{*$qmrCCXWhd$8gk zN!$ClLN2aDvPn>~vKp`T5ggQ1o}4TBX8g%(h~d$`WM;9McaNz6aBbE+u80;zLZ?*0 zzc5vs9~ATBbY6>s-NyLjUc#hK+0c+vkX;n(hB%9|$Eb<=+&R?{#a39@wqzYlsddn`E*f>&?E%3ra^ACh- z@i=yOS-MiQbpz;X9GC-7&O2ss}B+rQp9DLv#L@KSKV;=mMt12IZfbB_lOWsl}> zf2jTR$BDg68oeV3{Iz9i7DVSt%;isd{Qj@>1^~*c^#s-S8AKy&=(3bD1SsZH4|rYL zgbjT%XW%mT8ufF4@}3)p9_}s%DC)ikF?`dsB<&) zW+KUf=EpaXO2$NFH6cmt zHP-^tZq`6SD~J4g&)JK1lP>oRJh=x^lkcXT9@55byA**tkR)(Nj!LEAwLw~LwOX0N80GCIf7oK8Os^KqRaYX^k-0 zK73k~tAc&Z27F9izGnu%gZ=%`Mn#4nf{0v=m^YUI$5(LtyRZN$^TX!;NlguxEkAJP z>nD?e1;;65w<6qtXcGwZW)c20GB}{Fh>T5@g1YNPkXnRCVBxRk!>$3zqEt*HD?C1p zXGfmKqvoxffQY1uRL%e)ELboN<(Mo$N=JjPt~yjHoKAZdQUTwQ2G%-fCrg`GPLtN4 zJCYm8Ci@Pe?`XkS_}`FfZ#9a8VnYhlL)1VDmH*NG-=E4M51)&_0XUik$8ONNvf8@Y zpMMGn*o+9BtRKWG<5xzwP>nHr6NIp;KJ2lo=s4@1=t3DrTaGE7%?-l2SW_ThC-#9G z8%Gta&EG*?NI!Hy#LThj&=L|$b|7G4;cmS;S#6KSYPx;WI?OrywqQj&J6= z3Vfv{P#!jfLeS*+L&&l6=S)#N5%K|rFbR)G=UTv2*_3@TH5QiPu)&x1=?({X*}#aX zP5^nxRHb?YRQ)CiQMRLyU|(F_PfTXE3)!{@R$q)F&=NE+T;n9NlJkCW2nmsr&*B*E zA=#aVCL4mHCjLg6M>8hslGT?av8!}t9zYkD7ZYj3L|Nkzf7vuNUz)qgcnB@6$HCzh z$QiD$po}Ii0bfa6WIGb*h^65V6}@Ztzs0xCS|)_^*WkDA0f^WUDeCV6CnaZr#<)kY ze@1#v+5geKLIkdgR6p(BkWy^Em=NuOj5T~LAA-j zG;+2tFRE%HWQ}KGka#~UHZYun&|MD+U;gpA*K-Q;lo0vq?8Y_9q*40irP7eZ*t_5S zH#0~+m*4scG|7DBKuWD+9CGfW5v(U;kuaB8+)DGb=e(c8uSP|XW8sfl$Ub;zm2g0j zX0(NG)7_s`aa|hfV+9b;OG3%D5Hd#E!io?hUZdmk2lSL~9KXSX0jPHfa@3mLoqp_s z#6~$&7_nGom~edR=XzNBm9W2d*C+NzfaizeLs~iYGVkF#Dn`q|jyBmMTredoqh*FE z;QII6-wSwsCPS&vEop=*)M$QuMT6V!SY_SXG&#zNQu*;_a9aMGk zTW1{CB#^Y?7PZ{D#Tbr&+=R?ONt4Q((s06jf;XDmMBz>{JFN*$HT-v!2gU{{=Ks6G z1F$aK_WMUa*E(PeWeCGn6iCP0o_4IvE?Noz9U69#UFK~NuR$Un3xX9l`uDCGL z!i4A7sJ^?5e6bwj+>X!dD{~@=aL4Z zBdtcOU6(|gnVqOxpPp1ql2A(9F-_84s=tx?N>WP0(QQ)#6q!e6eq0%l&aUaD{eL1f zu=o0u;?GD}?1?u5QZwlxC!2(b1T`g@`L97JcQWyxm}$r1+^_lQik#@vHoaT;pb$7& z$2mnx#Z`x#;h^d+bj$AwByG*Z7>?n0%Wpz%WXm*i^GY63=c&gnrbus7g)x}ZZGcKS zPva;V&=4T_@SQBW7T%Z?+Kf&*Qr0@aL+sT;VSlQtL*GM~%;4ihTjH?iVnk|dTa>i8 z$C*t;YM0T>!S~C%)Wxx%ogRKi?5}iTq&@NsOD)tJ&Q5`b$DOr_7AgEu0fYI8wrtCC ze+H0J#Lx0~&wd0E6HhV>zdp~9Z~PtfA!XIBwUM@`h&i)v9~`x+;9$791O<3$B&jD| zi+>LSCMOj3w)k#&61i8g7W=#Xd`J!O6D#5KHHQlLLUL_6ORPq<(h!5@RHDLnr9PLZ zIOq=uMou{(&c{n|nvMCWWn%vA+=rL9KE;Bhu?$@{ygCmrWZ^DaTZ<`@Z9m* zH75uwvX743NyfEsG>TXqJvx8Sq>(1C91;<&v@vDxxB=6u{!ggK0O^$fNyTX@Flg|0 zrQPR>|4m>OVQux<$Wdz!a)yB%Gk|G_oS`}bifZ{Mz(!OH{!eq>yJ-(^~{ZDQB-9RMIO zQ`Vk##Rs%6LWwFgDHxloaVL!7ce6*|As3nrc$gjlp%j3XfeVs60)3@LEZg;@suX|J8FJ8;w$0KApNsDhV^Bv_yysR60BQxHZz z6dDJ5XW}FuPcsGBO6?uhZwVAi!;1NyBfUurYmq4i7_cf(k7-?$6SSzo(=z~rJMP(e z2mI1I9g{qLHbmcZ8g`U~X?m>}4Um2_&jd78Ylo2PYHvIAMzDL$oPjh|F&IRVdJ&z> zsQ2D@%Wn|rJx~rR0lOXF?d=VF4}*>0Kd0g0;$z?va)C_pliS=ibHMP-hT2(Ml|2yf z8ZG@v2Z94TpyW1%u=YRh&KV@x62E9l&pv%Fx9xb$6M`@H9juR1DhekeZV=AU%E5^) zD`Xx%s!lXf&8;3j9}xE&Qjy>FQY70jEHiDh%Je)FYmjm4n1y&&r?ObTQ2-atXkn6; zB*7)J11X5KBx%7CXAdyUFln32L(7QYM({y=`l;WGENX`~0NzmzCyOrI!SaIe@e{%p zHD78C!hg6U0E_U~kb{&bptPEgT_|g;9Rg~OqCBuu(0@)`fJlc3+kJD~6jCE*zu)?T#IL?*No)`V;rITghp!( zQOXwV<%B=yC_c%jznuaCks1+~(pfPc5I{%*FLIYw%Y9n0D%VF#5wS7MEw`i|RIZMz z5PNOw0-{=|co(#Q7EA?8--JL)m2Vs}Oaj4_U$?#Q-B_{gV{9foKCG)}E-tAOb2K8C zPunTXY4Wj|V||HXt4^k>GDgqi=;JSy^*bXX4%}REDfjxUPit%YOV%#SeadtG{b|bX zx4G@yi^AoGRz>4o6yWi>_qndoTC%J9FmTnXd_*3c_Mw+TQc# zbo#1iE8mssu?@$i??%ADWtKm?lUOaA_488;|BS?G@5njyyc*IbNoDTl;C&daW%v@))aEZAjrKDV{u`>~unuWlD-T(NgAm)>P%^UrJW7Z``$OvK9B z>U?YH^`!X7QI>L0rNoH8=&o>Ax|ID;|6RMGFmjDvHwESri`oz3MVrTNDIAcuZ_+8U z2@#^TpbWuk4Vtc5-}+pk*YQboFvpVBUx^bHa8t_jlg*8;j2iOjWkD*}daF*~^-wiz zM31!dx3~CZ@1&PCw0i!r)@WLyg%$;xf=BlJl4*+G#RGGnp69vUUsB7`pBS#`KPMuY zo143oq>5`jKlX5WwrBBl@etF}$I`2J%^G#y@%{WXaaAi1v?=V#lwYva{D5h#pjWvh zT6*Q#MR5|FfM%Gu{_{~05uM##T_>VF)|U#>K10qxWi>w_EVMUqlYt;BKu5H3>%7#;gF(;i8%4hiHVO$-Jv5?g2LEb&uImF69^ zx_ntl<>vB4ZS9YpP07x+iHV$C4o1gkwgbY>YYW$0g%Ubcm2l{S;akG!2*FZ{hVoPt z-^O8e6<2uG?6V}Nm#?kT5L+P4tEY*sw_YOOIIK1-&oze$VOe_h>LmBrsUbaaY3aivLr%jA*Tv|#j`j*JC`)bN8h;Ge zT5#(qcx`^{*EcchAA;0NiaDO)(i?sp1%p?<-$@+pmCbAk#h4BRRqdq2PFwwFs zgnAGANi*m_JqidCvH$Qi(J7yb&z7ctKQT&)h~z8n z-iN4&k1~|5^^HOG!em$?Vw5`&N<1jlCwcfaq2ywhlAuMZl=M>BUlmXGj&@^;DJ@j~ETHU&FRh5d*E$I@{VED;J68y3( z(?M9eVa!Pz_8RP>2E6LJlHh z@kx4mJg?wsr9dfVaQjYA+bpik4Mz?fn;`K>55admq6P*cAF+iSbEeVTwY+oadF9J) zZXe!ucAm7c1xNJjkj&Aa$vuriSf>H%cZ2$2yj-wYd;P8Lt$V>w7~D+FlPW+@Jn&F1 zJsEZ+71(SiEtz4Eymq$jYCPA*$QpqcHa$D*d|E0&nTTYX0-TN|a>$`MR3wnXKd$J# z{bQ{Cm&o+ooRq)6|5wHlZIn`-tnWx*Gf=YV2cbM%{U-YqeE%M36Ksk}NJxBT{65w` zI(p@~Ra>lZ@dO<-PK2c92QsYqx?UKf9$8-wMXw|Pt}dI4W;Q~s+gVqq)WAdo({P5z zrzd~xsl;^@4;r0k=t!gYJY+FX+P4guA{+|X=5(IWPQK1Z)W58r`Ct(-)~Rk( zJ5aLDaXSetPWpJC)myEPB5t#(HW7?}S&hUQ^-w&%h_*iUuQ4BKxdIL2C8oH4sFW}} zAi_Py-HsJxQdBb$0!%2b6&rAz*E&=M%MVS5H9U_Z_W~+FDWmVTp?jhh>bWKdXS6O% z^V`5|w2zMl35q)EnGGdL8jPPDY2HZ~k7OVWCsSx(4H-yK5oK?A#(ltb8qG{>z zs&gx3pu%A$nM?(Pl&-;;2ui#^Ku=PQQLq#OtRiO5pmZYk05qCMsWT4$$;I2sk81N_}zUwWNUE?ho6-&JvoFd)t_AX7LS zB9d3M@W$!t>kHz0dwaiN--(i8qa&duY>L_{8Leb{Te4b^&M-9N7hJq*yzvZlz!tjv z;g&S1Z##6##EXK6h@=1-d73+i1QnkMBa3(LXvL0%`AB4nDJT?Qt#lj@Dk<%ufvbzb z7&R*u(2(ro8tKrN#V|ueK_z*^Wgp37=#aqIcyTPM9|Z%9iePrCf!Ej(?j=q8o70dT z{;}|um)lnfIJC_5`fO~$N-(n#kvOm{K|APyia?6!vIRq6??w-Jhw8WXbXz7QTJ1RZ^+RuY&6GcQI z0@NetZJ0^a_@Qa*2YMAz^y%>Fse zp$U|FCsc}9eOnbY(u;|tcl&wCH`5$m&%*)aPl!k+dEu_ylI^d9pe?6i9COq)!OmC+ zu6iXeDDqT>4D|ASvaPzQJX~$2^CWkW(kM&;r4+9V3al%YtgV7#Lt$P5>6EO6;#nuL zhMve{TTpQ1F&-3{#`KC*6y-*dpaTmv-9;g6P@zG|`ZWxxLc DwV;2! literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png new file mode 100644 index 0000000000000000000000000000000000000000..0fc1410d5bbd7bec387d9d743c8619c1c0fa604c GIT binary patch literal 72757 zcmdSAbySsW_clr_fkj9vBHbk|-6frZv`7fjUDDDG(hUMq3W%a~qjYylcXxmH!oByq zfA1OR{ByqXjdK{_dR))*+A8EFYsI5-4$I5@}?6eRFX zSfg_=92~lcnYg&JjJP6*oO^bBESagu`=W4r<|Wd)aBwP}sR~S3GHjnj zZ-_={?mju2CZBYo5T2~`{H~WTGz|iI=$;wqCnO3#)wg_};6t3zE&73;WW67Jfb3&TT0w_MX)15L9E() zNkK?Wbdmr^BK~X-bC*c7igi|3Gw=E~gVeBrL%!Xg@li*#RP|&uHCyx}5p;4SY%5#O zY2?%9miZCP7qn@HG=cd?6AFP+lYRJN=_N&CpYw+c3FfQ$n^FxH%831as}(5 zmy5H?#0-mkl|p5|^s-24cF+WC#1is?by6N&C{22&Heo7>8%Q}*jkOjR^6}(A7~B*9rirlIiC>T4B9$;MR@X+`a2fCPkNK$tJN|}P0B2z&ziWSzQpkh z^CQQa->U;_+u9d8eEHsapAo(-ACzC=ALTva^NKlru$M10H{ipm^;GXkdpAQC$2`8T zk||o%{hT$$NX1?_=?0dAv9Z)Q6E27Y>u0?^w}^1NqfJdsEpt`_+hSB7l~&);ZWM*8#}5qkRG ze+Bparz2<{iP5VkPjKX#*jd@Q2pL-#zC8X!V&U6&jyf7~Y(Q#9q1dusDX7~-?kdpR zmjCSX(F=5H9kNXqhBgI%+kX@7VXT6`SgY#a*SS zaGZi)NVrtu$A(O_6129p9=8g%F19i+P%glQNDNX=#MH{fP~9)|QIpk{o0K%C)}uL& z7NVnPBw*yFzek@(2aOMnXNY%ZAbwu`=}2vzLo3}<^n^<4z1!0IQr^<%rH2QS_L%1! zl75Dg`yX3X%IOd3Ea;Bop2yY1ad&ohe(WUa9Oy(?uIjXiJ7vgJ_sY6bk$m3vA|UU5 z5rvA@hjP8;%1)+uZZ(FH_Zw0Q*$hz?(0s~lC1l08jrUn9tl)KC+i2$~!`S5Kk;0ME&xP9` z(z65=YGf}9oPXT^XjIrT+A*S4^i_Ygx;Gy?kTQw#jPhYb0jZr^tNTn$wO6&r4Eu~= z^_ZjZ`oJ;5vHelPCg;Y?(63{w8a_B)^&Pgbj@^8D$OdKIo9LF;~d*)+nZaYJZn7u{CNCN`D%%*g!o<` z9SoR0^LB7_>UNsma5W8IN$O{7Gt&N9>s;=%z^Bfu=k$>enYYi*+TLr~aJAWG;_+{5&+oGRS`7XrrNrHb(085T z?Ka`Q?7DsQmTT1fiTBp%m#A8WI^NmTx*E?lAv2*=k7`c^58istdd%zYD_5_ns|H-fk(5hh%ECgA6PPr zt&wp4psI3;*GFIc@hGgNb?PAI3VN<^PkT?QUfhuBT=F2RKfkPKuVvW35c#8-Jg(wq zk~P2Qz)`-WsANM=qp7Xwx1OiTyf!O-THRWyrj%cYOn>ySr9NlRH?`iICO)J)v|qV? zxuvPAa^>u`aFu}5^1g~X$_sjvSE5#-ru){&ckEpR+J3;hfb_65{LgwvpTjkr>!-ie zmS1%l?kpTOM4mhKl={iBCpa=Y=~NpK*d^Oe952^*xv0HS!&4W~X0w)Tk-g|FkGIiK z&~_>r%(GuK?=c%nB)`n_QrZc*e~G|X=#=uJ@P zOBV?}Dx2#^WE6lU79Q)E$pQxk@t+c;jxZDV^z#gkE`K90BMtS_~RAIKq^oPAo zUA3O;ZL%h_f=m067*PR z`T1ggMPFJ5@jAvv-I2%DVFvyO{LD+)+u^gX7kS5Be%|NTBLkVkbBD55=6lBr+;45Z zH#oUXaba=V-s;WP^zC*Dt$B{`$L#NS=eH^;2`gQTU6ijB_lzaj7|iY5U%|K^%3~P0 zr2*⊀+~_pvivK?hv_@AGi>mtHrq4q_i%dW!d7_H%kTW(L+gw9+X5Pj-& z=J(Myit`Y#B=3>+B4+jWwm7YQqT$z&MZ?ZO-6YfTvw1qC<;aE$^7fqx8l4_v{6KOuO+f3Baw)4?I$eUAVK7i0zp z`Rf@)@Co~g27j=3{(K_74}?Po|HA=)u2~3wKaIeih4}Y1}q8T zD+f-%l@DB6y>>JpbG5RxcHnarq_}&64_w2(W}zUvd&JQ~kU~>InM~Zq{xumFGb=MI zg%Bzk8JU2+u?e55#IwI%2mceKc;o13%g4gv;^M;W!oh4~Z_2{P%gf8c%Fe>h&IF!d za&WVDG;n3IcA)%I$lv8iymm0MH?ws#v#}@4ipPxVNeC=xXpPH;4 z{^}O!APejc3mY>l%Rgm%EW98xzxO?IMa_c`;{_9oE z|Gdh{#s2R%|8?iT-&A*aZ7*(P1?qGZ`p&8u$dO4EBTY5d1^;=M#1vO#cj5nGz081WramRLvEBD;;{J z-g0}J{|!rqK2DY5JyVP){U!5dU&_TKLrCxw8Oembo`i~sxC)PLi)A@|at0ZCId8cS z5)>h6x3wSn^Bwx?R(th&t4_PmYd`+V|JrSDlEHCWukZ5xc~6X@{~C=#O0MeW_!pz# zLUwu_GGBX~Gw<8o+hq^k!<8O~i=kMEG+905y-9v?0F&{}?(L1z$n0X2Fv*;@aj&!w z#2zOrx+5eggX$HhZ|;N{DnW57C5IBi09ICXNFd{LYF{33#WpN92uBbqXM-b61&^96 z@q!Ue5+PJ%aOx!ss-9s@@j9UhfHT%86ZI=k|S##aKAE@NC#u1uAM zLe--fG)sBdh=@t2jIV18pNbBR_Q`|L89({VW0DFvAd7y=Ph1_{;YaQl4F_#agA0)F zW(=O7gvYUkDz{hCxr!lBWr3?VI0Em%{d91D+9LRz(iau9un^8s=siR_iw+?hEtz}P zRs8kRZ$C_C2JvtS1S(}W1U&ZmQp6P>$Ono)wa83bt`$7S83O&o1wD1xX0@SGetb6q z$%B)KMzJCkpem|zZ7n(+Wza@+401Cc^tL>1DAjvVF-38F00q1xnVEBnAiL!_l?Va> zc!)iE4+%;Y2p15Z%@B*!uB`j^e-;BKL3}sH?QA~iL4iY`V4^}GM7mKGv@PKU(N_fo zH!SIt^!s}4OHI+DU}g*zeZ**19<*-7z(FfP>AH9Vs5@% zJuv)gaD1dPps#qo{Q=z;rF9?K?bot{9>$YDqkzXL5CGHGIR@__f}n!^rxq$JE=M(t zkoET>X`rXARnAG=ICK%PISr665=^SEl(XOpS3m$=ql#8T=t<@NQS|0JlpxmgW3&vX zFfLuOHa&;j-ECzq(1n2yTob0K$#~uv=Cjo^5}a;*`@>Wh`ISEs#x+>3@*T7r`|_WEYbX$6Y5TVBaS+z0L~geE zU6h2Z?4$;9^?Z+6KhayuP*%(ZEGA09jX(c|H(C8?m*0wWfF>Bo%vhBcgjK=hZVXr%+h;HU55!(9N0X*o~vp+lfIIDm?*AK}^x!%=(QGm4+*0 z^cFjXsGljX&U5vp>p3yKZ?5`$8!i@x;SYJDN4c=}Vvf#xarT!+d=@ zOmB}{hi~5G>!8pxP$=gW+WAfUh>+z2_LKl7sk zfxlu-*FhfjHZnq1U%zx;WBu{q-KxGRfna{qvuVK`ub&x;w+(AS2W&>X-0W2l& zo+M}U9a)m~()pO=21}37RRcx+W3uHK{XG>2+=Bw)J^ILBDHnY|T@>64A8%7y-IpfW zW+^&RXjQJQlwNn$J|I_5y4ux5%?$rN^MQ(V zJ(>(yJd4dOKLx(t7bTl{pc*A(J@oygecU8p;e^+b+-T|<|HWdVs6nKB_RRPi)`0Mm z196TleqSHg!_ux5;fL%ex}ZQsa-;nva^F0@Xaf?!Acs(1#ts&AhLa-@o}()nGu{KFG}e8ymZiI+?Wd%u7WVp&1>eZmZPwcp*? z(yEz8`FldxISnWo9w#^uZu{9&-Rr^>9xdM(SUZSg&CO2|i9bkMM>7eIi5=}G7UQc7 zoXV5bak88;H2u<*U^Qdm6nhH{JIvFih{QQlcD|c}^-CO|{d-T~Xha)CHi&Vl;9mGc z=8u(F0Hf>iXBeT%djA;*RNUZZ>$n_gSYLqARV!xmWyrOA!0_tI%w@el+3+W3z!+>) zus9G5CL`=0Mu`0snc@~vp<*waw*o^ap}H*iBfN=QkPMOlWya-=G42Y z&=3s9BV_UXK~*!bs`9wnvoeyhG9JW6X9o{(wou;nHCCk1IY%|GyhkyW2cQ8ClkP|A z+@=|BCV3kL1*<6{$vADc@4Z!RnAzUTwJ(L5b|PN`R|l3wr+AiHjY@{DiBKD`aj*gJ zkTCg0j9Z|a*Upt|j6*3Z_7jgqLugIvk53Xp$QzG1SRTBgO}?AAaR`IP^Sv+}blRDb z<+-X4DBeVn`#5Y^vK$Y)pMFd}^gW`wTL&odBJ|Fb!{Jl{Y0a- zd&%C5JXWrC4cdC1o4T2NBuC?`Ui>1Y$q%_Epyp)KLNekp=k%s7EkItlM!zD}MN_z} z7y!6+;>!xwQcH-TVE&Yy--E+zP)z&(u713!Wbj@ykjq7gR>GYd+$^%rFW+-him(^sY*K8~p=V#r?9KY(0OTDkREMb$LjYcqKBEE_A8X>RcJnBZKUi3pNsS;YU?)0-&L4m1Z3}kYR}Tm->$c)5jLp1 z3WTeU&bXCeMv2cO@^6-wv`=*vE19Zal}oiMAe(jZvb*I|&%N9H8Q)CBG@GDfy?)V= zO+99_+!@~s25$R;^G$zavaomkW*;sNU3_D%Ol;bHV!i^d(FX6^cd_C`*SzA|DpiKt z&(zL4xb>#CzJ+{k|MB}g@GG#TiO{(FtHLKlb}}i^=WE65H)N9fluV`3O<&ixsQ1S7 zy?Iluz9(P2i?3seOnbBGHPLB6VAPbrthb?S-$AW-d9qDB5w_Uw#@XBGio)`+RF;rqh*nifm#lpT73mXKH*kQTSy{_mJh~R%6-L zw;@HvnQEu>&)+9IV`77d8(hV`qFJue9eR1S+{lQDS9)UZm3PrJ1RUadb3?!!OWn%ttVe<Lo3*E8kc!QTuler! zm+nzm`i>`9d!y+>eNx?LlHrkr>K$PfgeuFlbB=fB7l1eq(Vt9dU{iR3z1RH`w;>w( zoa$bD&}j~(&r6i+wu_@7rD9|~x8?YqI#EBQdqTB(HO|&NRo}FapE|A&w#6`pO<(d^ zl}5)n*bw=Td8Jxc7gFkzE{I55)$ae?@jOfQM~A3Xn2${J)VZJD?`9ELnkJ6_=yd`n zV2i957@YNw@C}64l}!ryR#V?g=JZ@Z4HiyqFkCX3OGPymDX zIMVwJS&+^EokcRk3z4h9er}%qR(8FSW~5mv}RiBU1W$Y_7hF>;|*xq^cEV z?;yy16(dEm{_2eae<@uPE5QK*w{bfbcdsupNr7LnQKpcmZo^@p@Z52W?fc#_Uabj<3Wuw-nj2?8b{mZ&Ay2=)+{u64Dc;C$e9*~Qnk$GWE~~v{oVrvEY+&_G zX~WH1ZCzc+XYGsQG5s0WM>rQ?us9Z0_$M@0``lY|jci!Lxc39l@T>b0YOeSkR(sW( zkr$+HfhFz2aM=9(qj~_7=QrHxO-__h(T0o){ls}#`c@hSR9Akz_`v-CUqeOoip(XL88YD=(fRjbGdfn zv5hU2vn3;{YyL%;xay3Bg7c)hq)y78DUor+I*u{(X7vuB0?^B8iLh*gaAxSFhTkb);S{zz5eDILmr`&)U=Wqti1O;F)FhYy&7^1y+qGrWVaNjAk%weaM&y27FwmWX zspS%u(wnYN=k(jF*sWc8pHMFev2O59xhKDKK50WE-5e_|T`(AXMUOK5^jM}O@!8wC z3MvgGXqCT~WXsg|eQP>i6)iovnB3lF<5%gQT#rXI+UA2uOG;d(thmYD-{>}2(%Sb% zq-vZk=6!XI!c9w?qF6#@f=_TAP1N@)+(EvkrM+Lgnqz72lqBLc56Cn_ASNjZUc@OA znJw@cVep7Z#yy32YzQ~YjsTw8uV@G}n(<$%w&$92eL`CI9>}rgX(KxHw0X{E#-Zqj z5>6`ovV9)=n3Z=pdsYGO$oSFAS`U?zUKHMGswD=R2WlrYDA(V$d0Q!#KDwI8K%fMW zW=6yWCY_JXT_lg;RhFF2Q3-D4D3`GkeMDkvu8Nj=g)S}0-6f`Tmvi;VPuo?-Tyamq zE+t#KdN4NUc(4?!RLxT8j}_-&t%f&2NVW8m%i0m8wIi0c*rJL`$8l?X;JBp&NClAD zpBO>h_M48w7&oTWzLtg`aVU`Xn#&27OM-Ku1oxTpE>4=aD7rc2{j({C^y^zx;x`hO z91dx1yV-tKwh$%xr*ydmDb+eCm2Eup8YmRrBJ@l5sJky0Ss@%byFB)&e8B51t`IT) z;j>z~M2FAX2lXFBex3H}G;w6OEhcJ+YpUU5gR%4nrMu1a)3QO-)Qy1&B@s*xgRxneSQ_`!Rh=}l9gYaF+>?hCfv$Kg=Jf#Kc$D738>dv3R28M@?K%D zrn_WR2h5_I#Ra5k76+<*pM7gV?sB8<<+`4$8~FLH&QiI_$R=3>WF|Kk1#9h|Zz_jr zQQNfh4(o!z8aej)dZu6Q)Dl6TEk%em3m04ZnG#E?>nuCEt6yER-$?>aOhG*@SJnj zy790E0HImZRebPh)l{Qp&1}z1nMQbr8*NqRE|t_{ zJ?>B<1pAHQ!o?HjM*Z9Q8JRV!{;^pS0;(+UI&<$`mHLXY2#q&U=z~A&S?td`6%cnt z53D!19oLk9{A#__^^BD1<(~84c;9KOX_ebDPq+Ha*uM#5si{U{hEOWgXk--Nq6|(U zoA)ALARoRFwyn^I?8~Q#lWLbg_yCRjylhy{C?#@|RvGw+#ijdKhfIkz(S*bMhmt3T z1^6r*4-_@o`~@|ZN+0~HQHs!SCyP0lwtj2m$IORUmG;>sxH8MAiw=<7rJb#3AV7-=TfhZJ5u&dJ?M!`ucfL7dU%%br# zIyRLzBU%IsdBZL1qamUO7(MM}CkrNH`j^elPF1!c87iNSnsOJ>_H^reFD~&G>MwWr z$=$t%StE2m>Pip4Z!x6cctO+=aXx&=W|`1s5G-HMq}yKgRBQ*~Q(&fjz?hdkr3g9q zZ(RF663%Y*9s3QBy(ABWZ-mQ*Mxe3)^aTFq4*c)&`TpdBZ4idsU#tW`pAdkylhfLb zsB0AV+pp@A&sD^k$|F(w9$@_~Tfpc&)8=5$3~*wLF+l6!Ll{52SNF`GIy9v}e0e4b zJv^lEpl4=4Fo`}^1p9Bj0><$dIei&EL~x)&qeXlnOS%-kf}*vG%sTi#rtQN{ANI+R zJ%l}t#~!$1L`bGbq(hk7D;z}?4MFfC_p>7U8A;5z)Ve~27xh~D5xmZJAPgcW3$hnt zCP3Aig}k5Khqk`=(VeF%n__k5227)_&FgX-zsrODIThSrddy|T|GgdqbnP!zT;Hru zXq5SgqF{mm_`Q1(MnN3b#D0#hSRuT&s7kW=BVoKj>9>|>8ObaP;RuiqO@hXeJ}Ldf z^ozLQCnFAq8(=qzNaV9RpX+&mUIVSfhY8Gg=kw<R z8YVbQ!L3t_Pm(#^xc$)u=ve+eYR)zY>a$VEnSZHSE`NpoYlik zM=eB&!Si3=VNWDW*`1dqmEFH@)0c@7Kh^}3s<-nP-m1w=L zhec_C!UtOzuqi%I5qU7uNe}|@fYko<3?g>MO0_HxAzX+OzEQ5j2Q0J7mPksi^rfET z=BUC`o7rg$qakSL-davn5&Qg%MXF7GmBNJSOPNM#pec%y8x?4Sxm)-%ci;ScG}_bL zR+0_#)15op&pt?owZfcf^JuBQ{?Yo7)Np|cB_)v!SQlyuf3KQulH}plMo3XPDZY>( zJ>&qJoo;h$;%nR5K#uws_meI96i$l-JjfGd5bFW&_=)}Ohgc!zh8bD-4WF+L+(!Ty zKpwSzr+QN@Taj;_>dWO$P4OY7(3yyQrjVWj>QB#x%gxe;*7g|6k?j^tVF9bip1t{& zHb7O{RHTTh!6z+^Vp_rMY>n%_;m9G30o#FCPN;T;c^>uUpd3qxO~dtzZx&;U*9HhI zfep5$is^hG%!WQLv{Oc@nYmKl0BpVhfYgg0_l3@04FFYUtY*A+4Fa=*`zKR2VF1o+ z`>4m#C_n-?PNp4{00CACe%+N&SpMzJfSA;L3`7hjr~U!nAZR!3&{z1=e(TGxz}Yt+ zvUU9 zr|s84zz>WT<->Y%x&0$^+3mMAb+GLhn*>%)(o-{> z_iB-1&tiYGF@Q~~5~H9r2HV+fT@VIh#!~4l9)xq0WPmM1F*;tkLW)zEYqr=`K5KIx zH?sP7KO?lFVy-_kywm;yeAXVZzVd18q;DV_5Hm@48pXy^pCzZD^2vUJ!h)`$=g`Ax zW|s2S7`3PCY=5zBxbXSClj*93P{!7a&C*>VOj;~xT-KK1Vh z>{a$vzewCsx;cuqzM5#4fsvulPon@+_T!tQ-Tp>#X()g=&n>IhKNjIjBln9TIT)9b zhR^k63hN6T^^0#f-`-r$ynOHBFHi1v4(NU>(6mMzYFn&kuaw(e&GrP$|29&Ib@ZC9>da#zV&H++AX<+7#3uvd?)OMpS}5{~)uA zr9t3xzU)lgtqLq`o#6MkiIFFdHQxs2`b>F`po13l<_2TG43`-lr<;AVC(?l%eu zsny)@1QK&W&+b!v&)nz%D%U5-z*8cD7`W3ok~IFWQKZ7dz8qqh86B@UDXG~}O`Rw9 zJQ^^(y*_n83-Z(f(4@HJi(UGyWO%06Md2e(^SA@dNvysXMzI$MqIFaj1{CMfK@5K#QGu6i;I>9X%t`@PiZYhVFo*Qiw5v$|bRhn6~G z?TP|t%I10p^W;WMyqq>iA0){kN|EQV?lqu50bD{Ei@kH1s0q-xAbKU4tT!-MBk&4H z7Yj2kbLu}I*Nnp)n;`z72&SIEdn=N6jCW+Ba#F6jAN{JV451a-+>`+Jy3h~ zu7C}wR9pUKflGFVgh}#g$7yF;%d+A6WFzf1Mg$LrIQ9!lPxIw2T;7!U&FC{X?0IqlP3F*fdjZEhxTy> zP9THVXaZhY$bN zUQ5^12U(94s9Hz|2j3r1*!&NS1vpG+s_+jl`vajrkc-q{V`z96(e#202}2rXud>=d zVa|fdI|zLXyHWU-)t5oZX`lWN{QVA$Kb?|7GE5|V7XBz(cC^A`>|cOU{0`GcWBqM6 zV4(i{ME}L1@gMy6rxdck_a%tv2u0B-Uyc6-yM18Y0$52WOsf}$mR{YVCBT6|crUB= zAEtBCo;z6wRdW+2@p$~Fqtmca1sOO@U9hm?cP{C4Z&>;ZEG8)T&+4d?TB#U&`J8FJ zUmTOkCX8Xg%{ks#R+sq4lHn!z3O+DC%N8VPg9w z*u!5c>{3N0w3&gT33*ZalF7Pu|JfZZ0*1eZ$+Gyx=T?9?$iL5dxu23Kf!dp?!5}mD zWev;2oaGV_v*8Yfpla_$R(sRiKs<^GhU0*5BjcKNHQH}NCJhFTD#U3_Pa_=coGpOG z;=3s_6%ax|IVu@NoG?Lo#1C}6C`^gm_X=7IT*yK=M=ut}^fEz@u6Ehk(Iz{ zwpH3KwgR8Z^fR0w*82lG$oq-r2CS)8B_6LI7Wp^?!%&Ex=`!m^{knEKMj@*+om40a zA`hu{+N62ud93aW6TAoC8I^RCxNni)++5oD?(YG>&)1WQ4<<4X_uq^a1f&%v{xH@B zF}OM=SY)hE;4pj~L?YSCoq(niaysj-gig$-M&!Qrh2Unwy7uN`bD@JaQw=w$SiP7= z``a6GMV@(xT`RVfF-Pb`RLseF0<$pe7p7Tm`z;PDDh$}f_GD~|g<%<#o$ZR(q=LnT0qFgRhA_WfGzwy+P`BuW3NTtw@!ka--#D6^eocL;D<>3>}-@l_GA(mui0NI;%G5t{U zX4EL@PI*nw8Ydh{!|iqD`rt(mgtpqYkY&0ozI9O6X%k!d*66`>E+ur7O>s>-2|g9 z4`Csb%H;$d_NzMXTvx2Kmb6Fm_kZ?BJ3p##w#5#NXT3rU%FYJFCj^6QTtq&dPaS|; zDP)abrkS2Eq_HVexNVqWxsf{3W?no6XyiyT5CFt|XYZGWx0>ofu>i z0>eaUCN2ld90ZWDLhzcJ@HDXC(DcqieKE(TlW0$~@1$P;CZEv!}lXoS_=s;;4 z7v}?#*z1+FdCvT;F?zenfB_5|a4JvDB%3K4=b_lG7y?|~ zsgFwR8774j%=wQ%)K&VN6c@KZ(>o=pia|!CH$o+lv@5F)0n!DN4+^iAp1Hbxi7CZX znO`<>B9O~DR^0%wh?a?|s$9B_cc*fZ2!*!P9mLGih@Hoqs;*C_W~S_=^z%J0PZ%O~ z6<{&gP6QQy>M6hx4N^#1q{&KP{!`b>YE7oxN-L2$^2??y@=^@MPy%jOvzZaK?8##X z6;^WmSc?V_N8(h$7Z*`od*pBhOLzdos!?|UXEuL~oFo#iLGCM@>3M1GL1^Rs=5csURASCmq}X`j^CjE| zA3@-00q{kKmP3EY!dfnO% z(A{c617vD}6R1!5UECs?C8FzkFXW_!C^nNU_8yLM%Zi$bp^yGhFk|#9E4?R96wJ|8dxi>!1t5*AF zIl*ovUSnA10YYjBEH0Ll-^_lRxQMQ22O;gMzHbpnU`cz*Tl-69NBK$0Q?n0m9p&fO zl@RPG@5Dm3Z&d%0oBmrW|2)5IOl!KH0yDwu;0zm7Yssg zVFVl|!F)~|pD-FZ0MiJ?0l*m3>rP1pH0!hM=vULe|1q%75May|hXoeNKRaK|oY~6+ za@3NT35d}w08E1if*ZI$X5SWHT4n&L(fA+u@>jZ_!qA^B@qua&q{j#ASXW63elY1t zF&xu#d4Pp)y9`vCx}*1%YahZSU_cbW*`kcDTL41K|Lw`Qhae2@94mS_D2a{z+2U=0 zic#O8uA>2|Fql(Of#8A^<4ChKfyd7OzeE3{Y@I= z20j1OaC2ce=Xt8`rJ?5(ADY7vP{qIaGXfVDJoBj(>l*-BPV~LJt3(6!p6v;>nbv~bedSO0NR5)~dwJK}~bZO;PEWSr?FJL}_b3x3K%(IfQq*6@R zbv|H2_crK(1~}B{>T>^@L(e^NWmTa%VMY{S=AJH-?{>IZ&G z>|}w?`&@FZotJ;(1cpbLmQm;!^Grt#9P?Hrs9Ndv37!|kAh zHXT6P7w6gGAPQi&*pWfM3N+9cUce1GEn%e|0DiJNeF=<=GtAC2i2g&y^xA6yzWtr(}~#Czd?L8l}$H_RwS?J@bMQPC|^~u$dL$ zzOQH;75a6`x~6ayBo`W$kV-@nJySiMb6vzZtrv=^6Y1<@F^1W=on65C7_5j%q6{D|^GJfK1xFdG0;U9O4akx#Mz7cPjWgIIxAX$J zl*g+*^33Fm7>V1u_mv7)XB>wVN>2TuxWjdphN!+EckR;^xpBW-j$R_O-tcwJPPN>e zEI)-{YmD6R&<+rxJ7r+t<{L=i1!?O$Y@KBpv>1W1MQ(>H2Gkvn(pi8dcgHl<;_?dQ z(Civg3U*61@%OWKJdAbwTIHd^WH7h`;D`T~oyy@0*42Bs-uTSAbii~7_(Gd)$4eJL zI@}MQrD$B=2ZB;-;%t-bA-B|aA5t<7E8a=iv4 zu}lH@CCT9bw0hBsS$O1rab>8e#lFUU!8KJRFC<0#)4;NoWp&MbgL5Zhg8daVIfS%N z7EC5&K3qUP zo=vVY2r)248DjD5!(D3WmLI%J2X&&cF-6TIho>P^&U)W^9(N+v zp1^GrFBrjSKyIw!V3PzhIVJ6S=wc+lVtYsJ#yDVf10N*OxS3x(>8Rp16J$RVif$2K z0HQShk2$Xz!WWswxa^wARI5`3WPp|0yxPS^0I9B;Rvy{#`)7 zsJ{Bh#sia-=xE{3;hk5b@v6<~=`xe1ry9Jy<4dMbsjVn%OL*Czf0S3Td$&^4e*G%0 z0Z(ct=LT+jWl!B$v$^0%Q z)j)75NV-FtfQ2Gg)m(eP8B`~G48U0+N+X(i2vvfrw zwWHWwmpRzJ_hA|aMrCl>v-?0y_?L)a1_MSQ-*OSPg8wfbEV|c`;dmJ4Vl2+xZ6)=p z{RK=nUl>_dsu~U0OYk0>NeZrI`3+lFyCR-M^6$_Y_eH`&ER6!GzxH1K0LlSI_DsW$F(g#Ji ze)&aG{DQ{wYX28O-vD%dl@p}6EVjMpLy!97kM1J14r_w!%hu8cf_)H%PphoNI0fbq zvQXBzfNb)H*is5Cpq=adw2c!cDwQFNyu(?H|7CFm5J1*1?tx^yVWSVc`6IKsvIf0YQKf$3oz|D}g5VMnvL;bC#Ww5KFON)3uaJs%*|M%EnC zl=c?OhQJa7{^YOMx##OfYGvRA9^v5@M3b}_OJ~XAG@5^uBK1`JY}@QV_x5#Q7)l0lGxt!`(C_Db;v) zDrZX}>q9ZcNBKKp_Z0^?BKXFl&o~o+Qt?1^73z(le5G&HMjJ4}9GBIT4KLl6fudnt zj=KGf3X2$=m0a%!Cw#tU=U6bvCqowGego0-0r2RNRDQ0RszV(dGKj`>%ylx2vIwq+ zM?eCj=-_{eQZ?|5Q1U$&E;T$3UHYGsr?NJ<^1hHylw}|Uy$_NQrQYxUj35jHQSyHt z1V=G_4gNa13V{AXTUcA@14gkPZ?O3#a-KzqDa3|=Q#k%|poM%#B*`E%PZ@g-XtXWV zeZv4=i`}Kghk&%!z!Vb@E|&m6Ui{zhIFjQsTYm&;AXwDCFoc~aml${ujD+zx7$~iQ z?;qlVBb#Kt9^i!Kl3eJ#G+8!;bsay}Th1BZ3Z%8{}xo zNP6^w?yCfXda%fv*a{GxUYr2|NGE``ynyPJFno8iXrFvNt`FN6d_lG}TqEz9uMqVx zFeHv^Vl-x7vx{FGnwZImpgidbOgaaN0QXuWX5k<4pNM@MF45BwX|)aE)Io!e%iK-M z&tB(t-B^)ai2SbMLQNb+TVJaT8SF?ZQte;UKo$c|oZ@JGeu4+3Ly-A8v=w8}0%8g6 zWFq9TzW&_OWNhF*x_a_ck-&w)R0xg?IF%YR(kT|>b9dDCZB%zS0S8{ecdM!L4#0gh zWqrhc4u%?imm$N16$eB&fk*5pAgiexb_})lU6LS35K06S7xs=83{MKUqGO)_N2JZ2tZp@fjkB$+B?D4EHa z$W&$$G9{Vb^Hx3kdH(y^`+fI5{{Q{%<9Lqa`L(Qd57&L&*L7a!`Tc&b-~?If+yl^6 zNpUkAPzDHRr>w+RKvq=$Tw%p@QaZL?1%CB_8;7~DDPc~5RG@yJCG&l7R6a)7ggO-I z=wkg8tLF%_5HD;t{^T=hA(4ME_QExGQO5pz3-2e6cfsI`#mm8s9wf2H<1E2_%aD)# z_x@m{S3%zYq_*T@ImdsU)9TX1z?6t8umRwIAO;rNe8PuJK63 z1`CfT9&F*oW#b99)Xf2ZuI^~?5%70YI>DiYpQEougERUB4l|Mdg#^kpTsWj;4JI{8 zSNOZbbIf!!iWlc6pH0}csRHj;I&q+YRUt^!A5eLp;x*j&eCxnB=^8pikj`D^{jo`f z%^ zUlsry>#v{uw-FqP>m3RRv?1mBUBvvYl0zV zDdHf5hI7#Q9*9!Gfo&a%?T~Q_m4v^S_fxJa6Vf5qM=8SPNZlGzsmDs5jxCCN%wmISMRx|pxlQ~5Fn{f1Y>*htq^w({~pAwf1&yQM^=V^ zeWP&@eGvbr4Gi0xt8}>#rxpL#4Gh1&KGi=5wQXa)V1_W5VOHST@|qoPCm>No-1|r? zmwoEeIm?-?M(60lewQ|MOS~5d86K^Iv{x6$j6` z=sxtFEV?F}I2T57~bAJM5+fN-G>;&3_G9<<8F<_m8kP* zvp=J@mmHQEIx{kX*1xh#K+gO)?*o71&C)(p1cwXk&d=F{QSX{PVP9|hd-?hBLd5v0Lx#Tl*;fohXT6k zFjO4|I1w5c8qL?{yqtI*_&h9M^H?yd2j29|vF0k3<7w^GD- zxHE)R(qbV8jm2kJ$0in6+W##Y z#LWf&;`9F!2EcG1jQ{r&{(Dw1Bf(QT0!_xK;ue6SR)A-t{osEvXgt|yK4o04?Qh+a zN%CJeJDOH|gh47V?1{qvCJ^I)u?ZmMDB|r)rFAyPm@e^-5aHAML@W9r=S$FQp(7r63UP%K*Mqw~A(q*#M{ z2}b%+-y54!Z2J4@@Qb)aJMG<)EVCP_rBi}KtEA`YH`$MBUz~5|h&!}6*}I$mtBSTa zd$mPSMm8w(*$|Pt-NrdqM(@@)PfeCL(o@yo^ILqQwr4!9SFmJsPv+`^`QW+qn}qZ(N8k171XD zBrlLOlw?R-TrmJfurqcWNVxtHL!9|q{;TkL+rf{liED$Ud*Sk2`XQH^+e7d%vXuSX zv1U$dt{1GAZ2F=tz?GPG%ePXZ8VDw_0;wUXA?O(`v-SF1Gaqd5Sb4Vc6^$YGw@1!*wEjG#eYBWUD5dxJe4`x9*m}k>M>Q#9RyDN@u*!KADi!U+Rl4Y(ai4epr z(W|{&tJ)2k{&YX}2arJSEAOfHF5h7+HHiz!E9ce`{O6GI(umm1ymap{2+6i~dS%|F5UTGp9Ct=khA}PWrA#aH~Og zuDzx%dDitr^jiP-=5{j?nTA!NoaY|+m1e=8xfLYtAJ~4i89FB^dtOkNA+cNu}ObHPmyiO^JgxXUJ>ub zfR?^~h$G-~!*0^b!5?PI5A2LrA_2IfvDvm%%N6a+ett$rW#~X+t{d$?5j0v3qIyj9 z3RX4xKe95CCa$^Mceu)j=C=}3&hE0ad1XJm8lbvYw88p^obgykJdY_TQX2Y7tf1AY zR4L9Py%44- zvlm!RjFlpyr!?OEg3sU><%f?vR=OY`z{}CKZ9!lF|Lgvv3>|EUJ)x!-3JLP#@{Mgn zABh+R!`sP`p-7Jf$#h&^aMo@ktybomm2;@IzZDQEX$S!|3`Vi4el8rWXVnm8fJn33 z8I|d~$j3H_oGdyi9rQ^DnXIIab2`Z(r$Yo|p~@OT0#=o-O{_ceN0euuXc184#gLTB9Ro(jgWfEK>*0*Dw5G`TjB6(jyeZmhbcU4ipzURXpguIlQh?Px z&p-AP(>qNQ376^Hag1~d`{TG0ao|k0_}RUp@k{Jg0Ua z$$e|le2LnkRxL~%yODK3{Xe5I=nWc9v;6kb^bo@rRXruH+=5B>Bz4xAW*Y@nE@5g; zEA?gP2C!xz4X15{5HVV(rHr(WnlvL?sPL|XN2Sc^@^p`9n%y@Sn=2I9>oD^9f8k=# zCL}y_k;qLzM2FWE=!ejy@9+7C`X{tZZOd~#4DF@~T|F<_e|qk^pg&?#kOJ*u{(%G) z-{l7Sq00%xj=O;63Wt13{(Vr8sTYIsMEN@{*LlJ@RQL;~xxD*(ta`xV21N00Iv+=G>L%BoP<-w1S48??16YBO=`@J@rtQsXy`1n0-ysWLSkNrOf`bsWeSXEgTVXxP;E0SC2=1M6B%v8c(p- zp;s=xB;li9waDuNo0TZNr_XgBF(2;a!a!hDJmu`{7Z4z(L5|RAy|1rfse#`IG{- zbEG;)cI?#c?}oYa;^WlUKhH~)`?7hTm441idHrX7jQBfte)~bUVs%bEUbQYtwCN?_2?rWqk!!J>Cxk4&^*4jfC6Yiq)@p|t&g{8JCq!9 z>mV5}+5r-wr;BXPU2;YfN)AqtFm_sH_q0lPL-le(xCqAj$iP;wcW{k?VG3#|!wd^b zK1%alI$0xa9xL;}?=4ZOGP zqe(j6x#;Aux%k+sX>}ZO@NE&b9sD5uaMj9E|Gto?Rn<+~im7q@|sASEgGc9vnPXes3>}+iamMey43<>mvx6<7bO>fnz znH#ju325cME0sUEW65Mytf02}eX2`W=DsSK(|8znrI}j^cU>OT8Y)M@ti>WOEOHAe z(N`awxK|5RNO8xWr%oV67ef|<9G-&AnVTS+zgE8kw?E?6Z|roh)N9i*$8~9xBJy^% z$2#+;gOkQCpRPtFJ10~1^U17r#eQ4A-K$_a`Er7BxiOk;DL5wX_JdmBVNmRSY4ZU^ z+|6Y~3)WzsaZkkYQ?onvrUhQhmIwN&M1SWa?mhemeM6I}X{8P|$M3FC@E3!8aW!x4 zH+=^y`xp+-mf^K3?s8v{J>{PfDhP5z%)T5mTRg!MvT_P)xt}wlBw7(dho1C=``)kq z+T>5A*FD)XI-Zx0lZrm~7-(lAHd!_Ic}FhlX8&{*AW*Lz*uZCS#CKGPrt3~#bWkYc z^H4dje{e$d`kX=ic-G7&V2zBix9Xw%@#8%!KbmhxsbkjKEiB&L>dVhMQevDnxed8x zipb}?E6k^_see?g8b2J|E%{insi7Lm_9yvw3yeyK%P{9a`Y5?^sp*}Lrj!fXqe|cr z-rvH0xV-og#I91~_tHD2>M!@RIr*2tZSM?kA-0>EXNq#5l)L|!*~f8G*Zz*c!!xff zZnJX}zh1^nmbJxeEpQuBgRrmO-0+M?Ta;7n0eP=bnW7&d4QCmeN!km|L@>ReWa$6= z_#6ydEPKXJ!0=u^wzDqgx@g>gv~vf1yv|g8-H$qXJP-D;CKZwSvW!Wv2~W@qOQhs} zr`_*Z`Wg=*+FflBQ9%GBHa~;Z6WH_eMcnKwrc{~w(I(E_O08t)Onf)T=v_r-=&-~Q zqy8Yf%o{zl);#-DOfwPNpi#$X<(y#R@~Ppi&__@@Cv$~PhNDGUHSePC8jpXISqh}l zREb!H!9wiL*AEUmvcG0I`fKe!*DUV%A3R*S@z~?*$&Fpa8I9JMq?_uYAXMEjmMe1i z#NS;BD>uaioS7OvenN9nibYv~!1Xa1X6ilPc9^%^-5WCcxWgMBV3Nzf(BsRrB5tU1~ zw_(uGd>2gK=gpu}MpnB@ng~1q2}ceQQR2v7pz@y2jxd`U+zg5|o_<#SVHrkA&C{)j zx@~^xY=OrOl2k`F`VAwYWJ?iHsx_e*JTEO#2Pc$mK_RUNl{mbr?3p)FWbtlz z8*OHa;dqFDO-OY$NzB3WlJOD!xso(jrK6osTs5D!)=@rkEpHR_6H?D*3C?r&7_IEz zm73N{qqNXf8Q69*r!XoQ5z1kf%{6R)$@`m;CElHEZ&BbRjJdPnly z@#JItmomwZf>HDR`a8;i&+b7-B6B~xHynyQ9<21GN}fo;=u@)mMZ#*L!6|CuXO~_$ zu9Equj|9=1&W>($T=AEUzc`;&Nh9S-O}HlgowO7~bqiBGuqb{>QZ$FU}1yi zgjfoT#8TDVSw{|ThTKP6s-73jU3t3Vy@Wp!Y*e)ft}&N;R366LPUtEk2ul`M&)w;) zR!|4=S6Jl|9row?$XS1wd7L*B`dsejv*wIA7M~ zq+}>TV88Xz#-V=4*e6T&Q?mR?mv_k?&KSr`*w`M@vkqwg7@!;Zvzg1XF4^@;ZVm5P zHB{GfK>J?!b`xTg-d6IlaXNO#jiEDpUV-g0Ti+(KpBoBIaafrGGD(!xpzp>*HMAFq znt5tf^62q%qw=}!@!oxA7^zLu%MV;nRUJ!T&%~rYe?(wRj)3tbc9qVe}sorkvImP zAEoUf+H(O%s84&`Ag<;&)d_GKCZiUTNF@-jCgc2K5?es7Rmj2B{ncH-O;1Lih{o|w zQl%>gm{q-%ou!tnx{V15++zjrI!{=*p{m6tSk_KiISm|C4tdLa(zZ-T=6T%fUzKA; zm1<3ItvP<*iu!ZY>f!6ZxK-w74wEo0c}qXJ+|;eS#LIcA_eUkGv`2a84~i{|MatyV z8h+1fyLD1k4Yq=vN#K?7X?brhHZDH zXj`d?q|Ztb$HZ4o1Q$9;fcCEMn#({}$KDK!&x#%Kb*pfcY!*l|F7IjBU<>dV)zg`s%OY#cI81Fa}wYgeo8M8## zny9q&{YY?l*(xbxDBG#Nb%~`^?$OPYFwa%cEmf^Xi6$l8e!6kp-=cvs!KDIqmm6Z*a-chakR#7llEBKpY z5ZiWzI-*$?*6|j}x^t0PnD!t91Le`=y+W=B zYp1K#LdQbLr;&U5X;E{ip4z3cft1lEC3@?pf=u<(EJqSlN{#F6!JBVpJvYj`$Eun@ z-uK>{YaDU(&o88ca6Injn9%Sv{cYl8zQ^#}%m{t;B=T|#{l3$kQWM4q=FCTM@H3wk zrC}#b6$Vd60D(xsrT^KnjeoJx=lkl?Vnq4vQ4>5>hafJ7e#eG+nU|lb2vOi&r`5oF+ zc$^N)FuJnW)4$x!H(_@2ff_njy=AW`oXF{{vjmHz%Lh|`W&6s|8IziYdx1}=N1Xg` z%e`jx>|M1R4k-hz*MQVf!SWARS6PdnN0_olZ2^m;{MU}osN1Mh)hcf@IJ=s?1%jEX zX`WCuy_Q4wtl3x6-xpXK(n71gwvbI(c>p0u0DbS=!`=ta8M0)<$V;4q9kkv`?kU>; zyw+Ok>O(ZG@+K6f+3Bz^hXPFH0`z5n>Mx4AL09s&LKjpK9UIRFD4g}3I9W63NTLO< zc~}p3^^QHl7UMX*>PH=$q6591Ee|?%1CR7H6&>}ct+KR*;mRPr^R*xz&@GFt5BU1X zB!qpN`OLL*5+o<+l8Kk@Aj}*l?%CHGotW`kBvq5KzIGtCgwFpW*ZMo2?ORv!@O8PY zb(k+g;;)HQ$1+{YYicfBca)0iEK2Yj&ed$TAlD}p7dB|#C&jm;mvH0@E6!0>2!a}8n(vdn3UoFhIOj?22kfXA%+04dQOUv8c<_2Z$ zgnasCDucy?oRd!v1kz&Uf8c48ub&JJG;TSOnX7X%tz%Xtjp1*yiA4vJO~8i;;Ee~A z9$uq)aB zT9h9yIq-(rR$AhXO`3dt~AeoayJEGrctDLj%L*iNq`mfxj3TI9)v8>v(vjiGiFcP!*t z1N1s^7@~|rhG<*JE2kZ9*r|bIFB&JxBB>`8UN9ZHeK>`HS_aS)-RMLzh;Tx z;FpjY(?7NJyO5%|@ArI~LXocl?=s_MCM)fR6`9{9ma;$X=3?=;-F~ynDX9=!4wAAu z4$NS7V00YGzc~6Ex)O>YKM;I}fXyMRTLEYE4p;v+ma?-W;$i5|`d6T3K@^+gVE_xP zfKsmJ_ZcSNXs?Fy@6M_;r_`BCD-ggaz_JW4kbVs zif;h=mgrUk7SlahHonjm2_^$;xv)YOPZM6#~b*}53;Quqbzj`6pn8LUEP!kv^0q5U>Q6nYXtLHp- z7&t^sQhWg)XbE{UYh9tmtr0MYo;Vl9oM#8+y|J54Hi&Buu^2}Ikxb{gskE@exT3#H z?f;*!7fA3+^rTZm$FaB)a4LUCNY@{=LWc#j2{oc2ak>g>T(M&Cx8!nHL(Wpw56PRR zL$ZibYvVuAUHr`~Dfy7~?eR)S=?mQ8!#xJ+B;8e~GPe3=Fr(H(LMVUS3lhSRcO4(G zerW2EM}6@ZSD-p59s8z?uBV(p(nt#@!rzloCx|;`yMYtokm94yFTm|Jbq54|Di`I?fmMzM_>C6=>nO2Iwq-HpKgS~) zh?s1?=!X0!<6RI}Hw;(#LI7j%jgbbViCkyols60`r!24+{NYu)2Bcdm2KF8b_I>Lp z!ttzT{Qj<$Yl(V6C@|8ssBN@s^`_;%8Ck=fGDpOY`t2ahz5MMa; zW!Qlu<8>YHW7b6ZRmX)F{iH`;-64xS+4mknsPf zQ34ak7XnW55G1dy1e(jw59=pZKtJ(G4pVcxZV}Dm{><$1{YCqZt~jFRSA+_1I9JIu6hXVEk(H=FklBF-4wU{}O) z26@xZ8mO(8kt~u1CRZjT_97Snf^&`*9^#dLb-~h;o=jO0M3OE2_3Ar$}W>+afmI#%na8QBducU1cR2}7e zT_jH`NingbcQKXxpFu0H{4x=1o{Lej+_5fehO&Lo$~Qev$j#-k*c2axz44eQWN3y~ zuW*}|L(Su9^^KyvBf-W+Eh8}U6$WZj2OVjRbd}Xem14}X9w6UZK%Azw(}u6?O#Ca( z{4+uZiLD+W)iiFZPrqZ*36MWw0*18`gyawgMC-*X^h3Ya=by@3uo6$V0I*GIUGWRo zboZA#2M5d-K|Zb9hL7X5(l)iJ+khHZ1WY${5A(O(Yza}HfyUNX-H{?00l6yElvP>k z(0!`|a?%hazXj1uTKLq?B+Y-Ix~!?8NAVqB3kD z^XJX^M`!@s;zS(?9{FC^Y(rtc?gOQu2f44uf|-WA}ot5SLVabDVqxe%g20 znUiwg%8kT~GSCa;j)rd`Z_scB))!=x1m9Q!#yGdfs?`QKk4m!k#^5RdWq3crd2t4$ zVLyxHrtNsvK&6g^$K1HrF&!DYhS1V0HuwkDcfADm17Wa=aj>o*tHi%uOus$ahg#;b zUNnYZPAixpze6-`zsvLzJ1-1>S6YV#(ftM|I+3swz$g1ACS=gL{C_eK27*_%2NME0 zIKI|?aTE!GsJTvq5D!3vet!e^ObFcnrOBZ73>yfQF)imB0mf3gNUkwM4vRoWIL*oS zmk+f6ZTS!}PW+`>!OVO80b-+VfYlBMgrx7rsVz@It5g95Y4RR#j~^^WfaY@HYPrk8 zV9YhCZ}p$h@3>mmTIGDX;;B)eEn6fi>wcnexEO@x5r~p%3TsVuA&2u)vbck3c{s=nm{ZpX* z*o$!1)~M`8jN5eL^?Ja^#p-mI7M`1sM68EeyC8z;a(H`)-s4n&h>H=?ChLdyXT(qEtZ`~DnEb1ean+g`M-01_pOMZuN&I}ajkjlQ*C1z7R&iGG6@4tYH$_e>8N&hfF zc4+wSAu{Ie?ifFE6tY=?4U}!aB3myQm7Y2jywcP?31!dcb*c`N>d9i=F?Zh?y|hVO zg>EM4cu{rp9AK82CUCgQ@vv}YXl1y;Q{O&NR9nazEtjAoxg_K&wbe+YaELWpC z58GqhsoDK?Yf2P6ugV>BIvZ2Ge9FfUi&AK*pgaD~voFy|#TOZX^q+FayOI+Sqxh}R ztR)oy=9deKS-oE3o-?Arkm=e$XP7zC6!c@hCo5u}+tZXel%k$d(xt%D>@oV(HlhKw z7k8A2u6zX{)X6849+{dHlHY`;e4)IOjkoC3~a{1M+J@{_EnT_J9wxB%fZnJmH2 zg^U9&P8`{LV{s!FLS@SUauG`Se>x}tcE9ag-mk{jhfcqL6aj4LJWs@N^zH(9Bk6%+ z<9^A_xH%QQ3@wybaW+9vK^eC9L9XujKWUI+*!<;xL;vhRLz_&(AuT@45r4`keR_!b z&IcW9v*QI`tz5Bd!wjc3VOO4v(>2kT6zafE|}!YnMq*a5cA5&4_Y7(~H(D)P_TcoX$q zm9S!kyC`G=q!hGZ3o~>wivmPTx*zTXE*CNRlf-x~Rw!p>gv9^&?CZOJJkX1}pXulN ze7{z6%bQI_3x?l(lyg0S>a$PdiwdZHUy1{pT8ow+pn6G|ZX%-&<%1al?)pebye`Ybmo8L|m7gCZMkF5?Eeq%VFzufndMPF5&43FuT%$oTboXilC=%{-}^Y!*j7WD3fn%d>awx zJ?RL!4qtg_dc@ws&`kYH!=(AA%N$4&#oxqJ_<1(;gT+-d_bEDmmcZhJMYIyu9pqem zzOaVoRgY$sMs^RH4xS+ay9bQUzp+P*(s`6>g0_y3QF+90`gu3yo5@d5@jLm}pWED$ zf7B_^`*Mi}Aw}yU19=PTCFm~Cxqg1VP02$^Zz`9*BNT${f9G6VW`xWR{;AYN)3n`f|hen{`)WlQg}p{XJ&P4)9176bTXq& z&;b;>%ZJclHMmbIWoo%DD6;QUm%%}7Tk(B`1B8@*xC&IkH2G6^iMB_?iM)(Y2@>3)eQD+)vcPQ5yg zm#eGiL4IBB2S2trt93m{Z%m%+>65OuXGB)B#>GdGi!4AjGg-JD!(3QDNWgdwXL$$? zOTf4PA?d&SF#k5DcNywC&c(uf#LbOj4z$P*h_@gAIY)!Zn0(_`n;XAxcY^&pWUeb< zt1ZzrgSm1KO?xz@0KrrDch;#L;iy3M2a$b#?T)9WevoWK2VJ};I(i$ngDFEOO2fky zP9uQLT5yeI#i6PHwAg`Xe_Cu1?f<;ymuf>B$N^`3VAqJU!)CC@-kkh2tY2N7rrRCB zDC!9G%nO4el5n5=zt38@{$@{&Ato=j_&DAuDr=a_AJ%G}Bd|A-wHG8FhapazQZ#P< zNQzm70lI>OhO4~zN>e40xEPvZgGVO=$2o|(@cHr0JbfHI?L(cKv?vI%L^goX{5nok zqJ;%QK063M7V!!c-8G1XfSf@n?Sl`y6qa&248;^Rf065I?E zemPBZvGm$KFa(-;d;wQ$R~dd2)`lrkH$A~~vk2(A+fYfkCkecn5y-er{n}W1ZY%xs zOTZ!kshz>w?}S8sL$I65Fc3S^?jZWG)HsF{POivq5F5))Y?4%lSjVyvm!XQ~iKf@y zKR$7b7xd;NbBHw0#%`MpLBu@*X_&8oJ$4p#U$NSNn5PmV!wOJ2mi4+j-daGCG!CyU z8%SS4M)(F3GTyx$nlrD#r8WHR?$tqLi3P#(g3tw@I5pHv6QAL;V)fm2K&qaGd3wWu zsmraq3aK;e`@~7GI%yK36nx+bV2LQ$1Qf}$&bANmP`LU zWlY87==}+yQM-04(a-s;L>H9DnkBQ}w;FGJ9GHtYKO~lV>CjE9A6IHW5!5 zg!EQdfq32{F*gD7(B#w^#_L$=eh|{$gJXUQX$Me3NL|GVhNh>W; z9E)A5|D4+0@_Gd=oftiCe{zoVa$@^s`&-U^vFd|600T;9bLe*G!_Fvi?Ftx`)TcnX zNRx}W%?12cXhTwQ#hcs+IQ<$xs_|NPHr zwvp~1+Y2XBN_&f=yot-g{|tkhbYMBXau!|6tme*)N}|Hb;|%0ltkJUASQ%q21EsLE ziZ6G`^n{!_A8=cN{%TU7dQI0m`(85lU6uMKSPOsKo((UQk$|XX7zl~$CD`tP&(^wf zSnpqJQ5thT(ijd)9wT9WC;5kjR1)ijLsLRE@@cexMoH$s^U9!@|Ft0X9&g#Bjo#29 zezhKSU+}Mg9U2WK#{GR<#M|$1TWxcixB^E)QZ~_0mRmD<76RmyR6f4=?cqn79M~0O zW?#?d6u0rPMPNRikJfPcO2TNAI5{&kivbeA!|<%hpM{U#g&ge<*|@_$BZ;~s_r`j( zC@JT(cha5CyqqcM(wCHc3TzFQ?p&IPte#hXvzM+ugD+Uh&nk$waztQ2Py5$9z30wQ zmA@9pB0A;NxU-)dWRISkuNG5%6#w8n_Mwsix^Jjty7?P0o;*^`rcIX4w4y0df94WL zkFhxK{iy*4btwwM_r!^60u`C#GmrJQhwmR}C`nR51#kZxQo8BwDnAWL{ErhbynbK% zX{Bty-?HajVXBNd4Kz#sIJ$E^4+|eXHq2OeG;M0B0W2l#HY%3?>b&Rfn9q7XPXMm% z@0@#uj2;WHdx!mdJJ@sl`Zp6<h|)u{zOl~ADRtt=epF!JZDmAXpeTGDbuz2uil;iqMB>`LTu zcdqvd!dG~sWq#do4EJeuy0N{T8tyA0-0AaGF<9^!*Pm%+m*1__rsrO@=@6TUTqCJo@x2PzQ)h&iCX;ArK;M!~e^WV?A=$=9?_6gb_qrX_l_cI? z`~eW%c?5eI%T4a;UGls=I#Js5<#zE@)fdNIe~(I4RnvRur8J4LKpJQEQmK;(^|?@1wzsu=e4r5-i%sLrJ7I}pb`YSe<7EmysCrPUUgag~srwpL75he3 z!_ZLbd`yAF%F*oJOGeZSk7+XGmEw)tUP6}TDLVFiWyz(R0D7#Pyv+A>?`N&?`dMN# zujQbcR9t-%5IL6$E5Gulh@A=v!%;oyby)4J>()QNOW@^^N{sb=r1~%xqKPR1_j~e9 zP6Hvhr^I;COG=enE|ge(8I;wAzx*?ALUwtpo2#%dg*8f{f3$@stoD88l1%Id;~lyq z2qBAIb8@;f8jPl<`0lKGFx^BzE=gvXEc>?j zncmPeaad(52s28eyD0y@O{DVwE&L3We^Wcz9hQJ(P2wtqeNM#69JmaJVju{kLjvvsO60?n|L(&)Xno(d z!XF_Am_8mjaCqdrqMV*O>n*#R9B6)bvdc&uU-mcVb+5XOfr-eO8gG=w6M_Wh1jHXW z(DrvI>OPp9h`Gk2t%&+9|!_P%&Zr&kITw&}bMBU}ketG-Vj|ge8!xoe% zRn9k|>J-BR2d11KS`#O5BFy1eFnjCTJlr^m^ppFyoOB=%btf#Ix_QnjDVn740D%+A zs^l|NG}MY;`|j;_3|j5!;%@5@qf`%I;wBQpXt=*pK}g&p9%~K1G(kbAb9;DN=Al>fkr^B4#AJ;yXDgv_5%9}7+9!7X)3^)w9cu0W9EpBgl#MV1YQ zgv5!3ia#m*P36sD$38oV=7N5>e0k$7TSj9zrFDb~*mMk8rEt84+R}+J+F=%FdvR;i+$inolJ#GUy!n z1qca&YZ@+Q&FE=d_q7>i?l_+X%U?GNY-(}Zc4oPX)LyP7G<6|z{ern9u~%hnpF|R zz|XHKW|aC-o8E3y zF{LFh>XQ5OkW5xbgt?wnnsbrpQzNdA&w*uG;+dPV!F3=}@I0^5r9oV;d;ml9SZj_7 zMGa4It}aNOFkFim_eOvqFw*3+g*eKCi6BtjepPHI2zL-gi6zI4Q$n*4LxZD-{irC3 zBM!)Oo$@s&N1^c=X}2+@H@ewvZ6U$^>6z|9NHc1{;mbUQC=y%b?v;tPg1dMZoBUz;@m5Cgv_ZQ*Bi9faukfF7Y?e}1Lu9u&> zt{$T955hIWPcfc{htOTj@TW8K*%fYIY={-lLnCEfo})$GhsNZlv^;pf*CZc9c>*rz zzFpP@O6}h-7qJbzF2T@=iAWAn!d~O@Tul#(30mg_`Y1zV;a{q+dQKAa{CR5-)SUL| z>m8g{@cw7OdC#$ge4@PJhqH07ghlfK1q%NIC_-*OJjt+J!eIj9O!&%4QWsdo2g0E+ zZ@HmGB;W#OS?gqo%lzT2jbvIrN{#+zBBjLUPLAZ`*P6W(@Sx zFYs{vC5czz=^|ZB=`97ku5e&Z&Yyt?V4EiB1tSUrhy6w1voZn+4fz9v((Et}BAY}X zFM+vj{UE-`cyCskFndyULqc*xlMEeR`LrxE->idqro+w z=?Le57x)J>UD-p~X9{Q)Wbhifs{;+g0{9NH?g5vdJPzfA;Z`}6A@3&YdrTd8Y4 zy%M=xqF3|t+{>@Zs!w9&Kkd4-#;T}J`W>;*03diKyV#X--p)q*F(GqEqV3 zPQ_CHjLn62-Eu(;yPKW9NyCkij5L;o?cq=sOh)Z~-rMpOXX{6H2%l3&I6;E`nVWOR zEyxmKN8gSzQ&3w?427R!1ZkWm_=d%ogOG%87tVaKIVJCkFnkT3!vdxp{mLbxFKqpQFX!=1Rd4xa`l-CIt<|s!3Cd9z65ceqG;lJWhvT(+6xNfdR{`!>|DBl8V>wpB?dGJ`%`a|&uCTI;gz+LI< z`Mc3T>;7hOkzXX-XK`)+WpQCSr~fjCK3aU*XAU6^koD={(qHCKX^AeUInn?Oz#RHd zY5-&dE`e?K~C>Nh}R<4LPj|JRn`e6Lli9E{M|w1eR5bexfB z|9a>PBsW?!2}pAV;{IT3J{1(uk}x1040toJ1ysG?*OAwv|HB4|oH`+AHe7t_4B5W6 zfm7RoP$TI~o*QGn69GUMo@rq&r1+9wbValpJgIQvw(`Czn4h*)5Gcjb5-#JbCc-T*9gKjvrMvn}gdWQ)Vg^%O?UY ze;%^m4VsMbNE;sADwTehP2cuz^fAAEJtpNxaj+&P*@8jmQ^<~Vd(MvekCU`|%uB7F093*A*^f z0$3B<%{Y^{c66^3Hi*}ps)`c3>?2a0P|NH%6*4olX1VfXH2Mna(dVzFb-TwKr6(;~ zp6zl*pOr$-Vi)-O-GxO`_+;;+=7P3;CRpvG&ui}n#f$UE+Iu{4&-gaFwbRjZ(8}QO z_mzp?r+%q{l?5G!VgYR5B@i56HYah(`pw-tv%Xs7-d|@g?BwnB<&OEvq%m=-{@S_K z5i^$L9A}U7`Qd(00O#Jb^lGXt-naJkDxcsA+rayHQ8C{*`>0<-E0;{?q+mN&v7==a z^0zl5CW<9Cj5i;#rlz}3cGTWxt=RNf$Pr_GyGd`!HZ5oyfG*$pzI9P7df01WDY`|p z^YhZ$=Z9am$%>Dzdd_b86iuy`*4DYBUa{3Y4xgb)_U-w0#ra|1$XZWuJdf44rNYh} zJ)A+aRY%OpX+c)DGdH_+rXoTVAGl&R)0szWFFB<0(Z<3*#uI<{DP_!*0OykY%0059 zVOOs)-NkwEWU|EF08ZoiS0Sn?h$>_hJT1(OwTtyO_vji2oboou1I&0UDZFCJdt2i}(#9#DQWSvvwTu#HEw|}jgg4iZ5W3J3J)Bc(Y zBj8}NIehEos^u*5zdw_DFeMEcz^FRyzZ zhNpg=y&C`7e)!62qr*Qv&6x|PE4dOm!e(1(c>f@M8Nm60Sl!?P?3d=DU*mG<{u@%p z_Lw&YS8 zD`uy;8x6W@x35K-(buKQ@!cue5bv(p{4`71JzP)G4MNYzR;7c zuH1RL%T|fc`$fzMRv48>WhVjz)LvPYN?_{vhe3$93;oG61C_e{1HD|!ZoJl z8%X_9vI{+7fj_nHtANQFz8p_#(hOYWkq5s2dasw{mq!8up;-##`e@8z+T-`zkhxRo21zLLkCU zht~@@a)W~Fj~7D`m;V|6HD@O&MjCX4y5v32Ef}?sqj6I56Xf{>z?dzNNf_RPzV0n^ z`CjrGUX5ag_}QbE*vHS37PBwe_qIxl#%%9nXH?K1QR!<;ItIr1kAGaD26aB{UhuvY z+2ykKdLvebG@y*Ik@R6&xWWMLgY!*am#!@J-wuvkA%C9jbK?1&eOg<^Bqq&2L6Ev^ z0hTpFt~iYumlU3B*e5_vhB?Uc99q4Wsx&{07nI`p6?IsZ(1eI{rn)6mX|CB$Tbj&v zyosEr_r-TNhy$lY(@&bLw>*&N>icNHFd2G2<3n@Ht?4?q_stYcpDKh7uh}nYba=F< z0=bJ}|A`f&DT6w}%9QHZV6B zrMk|999?T`8J?R;3b{$XoTb=FCtZ61;aIii$;!^>m_N;_ZAzH+>)uMd%lqEUUvp>A zs>p%}W|^k}T}poPP#BdghQ`^!CpY%E>RtO zE$4YxS&hGOTcYn@^OfNi>P$Xr#s2##=OVv)axC*rJM@uSYs9XUeKDY2M3jh~$db#j znHF227@2}g)~cyUZ7H*&yjW!Je_Yz5EU(?*vTYWy9FoHgz9D`Muyhoke1~X;T}JNo zhsTE;<%X^9cs|25OL^U(j-qz z`gz~U808O)*OSLgC2w@0&pJ-3+-b&yMVE{>>wH# zhApb5jif!+zI$FfH4$;Px&3^NPdaC5j{!UcnG4O6r>!1rGN;lje}P=T0mw~T-A@}# zn?Oo8|4LJo_;WHzop3x-bo;>d$OeIqUxh9IfKCpj2#bkkzFNqRUETo;+w`T-Jir1^ z-eD7D_cum`uQOGwoKjALx2}oH88fkwR^CK^3lF=%ejZ~ZPBdmGs=|gm1C-%)B`~&L zPJ$YQ;s{Vw^5G03yggbc<%uXEzRp9S`TD?FSqHh~Q*zN^AzcxlL`3Xi4RS}(mRbER zkml_OT$Li3)JVHn8?$lH&B7Gegt`ggXH|wuEpOJ!n8!-DZ-$M=dyVGl+3bq5b=1R% zfH`D|?*)KU=MZUEzQ5!j3TLFa|)Az0fuTZ+d#>UJ$ zl2?3Szpm8vCy<w^W5xjZxz==p6)a3!3Vc3w0cwmE!n7-ZtG^YCc^ z+s!8z1HK85kDtM;E?zH{e=4!E0=sBNJdxVp>bV8F@q&*=M21wYB_Q+ew3d8a zxs%aBg0b2)0rRGe0gRPIJWuz@Wxz z*pqAu^4gdM-6FyWWEU&joEIN2wek0Y&g#C4v?eI-W@aClYl3Md8T!WOPq4>FGB|8| zyQS6i3$H4kK@|`eKi-F>yxpNx)Cq#es5jLjYL@3S_}mASx3G0u-`nUi?E<-Lq={qC z?;UyewK+nb>rC>YtM{MdyDxjcrf`Ifh#f$R7yE-*y1P#tY$hYOzR{QN*Z|9a6)IWT zBLOtkmaSqso+%rpumPZ*4{Sw0%UIG62Ax;>E6AQ7BWs{sdp+l|@|ejNHVqL2Rnh;$ z+Np5u5P$K!E7T)UinzDohkhbe^AxtY@%n(K^D zt-;dsbaq`QOD%_7NA)abdej}tgf0l^RN?iy+^o(g@_Aa)W-2HY7Ntq0mi?T6y&(O% zKyq25!pb&Hyx64rnJZMJ-1(-jpT#L#=~8O9zDJv^akfdg_WLqZ&mFWvQQLJcURFxm zV&z<5q;PD};f(81HtRLR`#lDq}HqKXRgcg-N9UGdQTZJ@?C*cTA1V~ z%;{;Q9lk2*d3^o8EjwHHY0J8l6}q5kxmRQ?-&HrDTZ-%Yc6^0hCU*4QUFPlA@!^;)Xl@QlQT+z`*4xL#-D~J)x?>jD+(V6SttY#vpch(i0}#u$Ma-3Dz08`>O3! zt{V9{e6U6T7}i>P98JJ-j9Rl4H%VeUfH{soI3crD?9oL+PWltpfWEWm|M z242$D?IT8xW6>Lde0)-Om58({^JuL`BhDeKI4}YH%)NDk=p+Ng(Nw& z)>F$`w6%gs9xj>1=>$rYW+Bn_jD+(Cma3+ipUWodS&W18YyR!009YD*>PlS^BxHme zGR6F5v!0c4=($@&2l1pe5bwHHGOZrc6KTJ_5g^CE@!eH>mIl9 z)qQGg&nK2%T)XkmO9M8do}N2_;iml)m})qsatL^=aj-%&IE}c9kEwf+X~=U%N6EHK zKhz|eY$>9u8*9m305=9!a}M?iIM%*)K=Q%p; z6$RAtid$u0gF~C1If<_fs)~IaH<`oP}L@yl(4kaAaXL{uG zag(0l<o1W8NN9~1f4h(Ef{-{u`5x^QSO_&Wo z8LL8&1oVuLCykacwNBSlDsz0orckU2AVChNdaSLS4tBRZ zb`YPb6AoXGf-XNXoPu_kQ(AG!XhuCoJ<*){=DHSX_ zst7z~dV2{bc&tkb-u|?agmQiCdk~5BeyJU_KK)jc2-}m@QIadkmgoV6p#DpzHt3}0 z_WW`XA)Q)D<m3#Dz?425~hko9_;(mNrP~XH;AplgCrw@Humf7GL3n=$Z9(YxG2z2WN(+n}X$o!kA5*FrU3{Kh>j}*)bd5#IJ=8 zhAKKa@adG2ZvFEn&Y({Lw&ue$YeCC@`x0tb6BG z$lH@=k(n)$#eY8LRzj`w5lhZHK8FhcRhyt z2T}dD`RpPz>h3LY2mM&pEL@50RpY+~B|e(_3caT{6ic@(n;WW=uTJ55B^Ack@Z_ym zr!QEwPeUmp1u!D*9iL7Me5htM1tV%I%x;;lrt3aKf*FsSH(16p<-M%0MY+-9d-Yr6 zF6H`yKCwXg?S;l?#6#BytF0Mm5SZ<%q^`clK~JrP+8Ne1N1p6?H@T|8q1%^hJp9Zc zMlNvdnXV0fH_2-5Wm_^B)otz|jbLbUztKF}(Xmnd4LKlZxA;^}i{u;%KF;B(xh}(h$@9!| zuuUy#_z%z)cHEq~*&S>B;D~hYT|8-aO7087_UuZLI-W{~uHl%Y>%2qTQ4&+Ms9kXW zXP=)PHANJ)r$sNbW+6SbSHZgJi8)YHh)vI=zP}?kdt0Lqwdm61HacQFI>;U%y#DI4 zf5!Ua%UtfWONE)B?uz8dNw~+eK02hS7De2ep?CARG>%=_d!EcbW$sjK&x_2r*Dr2z zrg}y(=S;I6uT@T)t*@^-;yT5=-gg7~oX^?}sSurfj)xcOIn%PPHk-lYdQv~r4S$er z)MNDCMm@tz@^xD;l`vMb@jb0x*!t1o=>?U=`@ z4-N%1@{JasTWC-ANZ*wD^z8Qr--V@{8?;{8L*wntbiB%nQK9Nx+l|BI4AX<`R?`_n z^EK)2OR+~>?fXwc`Hs%-`qMJetHVR}+4jz|e6i1^FjgnUdwT>E&R@xhi;wgw)h;3V zp+3@C2d(i=47#WLZ%SGmmp(|~CmZL(uJT4}r{ZXAH}_AKLmd4qJM{4@IpoWff-g6L z;zn+ohktj9X=BAJGmXeHSgYpKXRBoI5V8f9WT&Il6Cz;Yd{=#)5eL`QHpGHy-dHGz zc(K>jig6@s+%(_{E8()``K71zKJvJA4enu;ZnT7V4laeiIXlv>SrjcN<1RN2ZGYpq zE|+KvwniUMye92FzN2g^7pzA=gQ0eII>nfyyG=l?^5A59R@_-)8jBYfh-ywy`j$@? z)^*1%-WIeusCsg>%p_jqu+&Aq+Pe`wOLeWDBy0l826ev57MIBf1*2uT;7O2_9(;an zaSnjb)|cY;x5WQWkYu^~MVdp?WG{(w#DSaMGV@V3OFs*z0GOmyA+!C;so{Q8f}c(u zE6g4Pd2lNFklypE(Lq$+cnw@AP|$sC)Os6sU`J8vBV<+j4+}TD({iX@q=RC_gR#!X ztz0Pmvyt^zu1F{N09WV7k+4#|FpPe&;Gr>XWmz(;9}(`tx-;^MOK+X+&8k*^2%DDS}LH*+7eJ5QO3o{2Ty$wRUiW z)cj1&qbYRoVE`0@P!5Qk{7<^6&ej@;Tj7ucxuXAeAL0RtBS_Jg5as)2(_72B@+bZI zU2O2XP$)4N4%c!NjZ7JUm*E?_mgmreKfO#7{ccSXG?p-P)0M%zx6Mqo>7I{E=k}EM zD0*0q2WXfKJ0RE$5=+E^5>YK>Y@fT9_B*)Y9*m0@K=abMJ4B~6p>A@$IOc|* zBxF9>b1xtb@<-q`aZ;z-aQe$aI@3R-)ddS2-a;g0yc@b-j{R5r6r~N?qY|l1tvQIA!4BNf{jXr z33TEGkg1-7h(f#MC-BOW($ZlBjxRLVZhXA**yc29Dc!GUmJe`{k!^EqzTi4CWCaIb zs8K%HX+KVk)viDgpbHm#p^Lyb%=-wY^^pn7HcKJY#JwNVEKgrS>*hncsNw1x%@IhH zJwZzRO6wL1?F1;beKh*6jGaNbx9o61RL4hyfj zI?^{bLmZ<69OHIrL_1V>IWymvpLMEgf`Yxoks}aNV7J?6LEW4WNh8P~i~eEp(F2vx74k%Tiw9&4)HO(ZE`aR45_|+-x^Z}{uiarz1faA2 zSGZj(o` zU_fj3{x0>I(o~% zXU)~aZL6)mIX}f^z#6cc2K1b(HPY^2mgyTRwTc*`7rq?5$M0x!=@1z_MqFKaWw5io zt<R9+04dgVtloi2cs++&j?!O;?vaoT;L zWsZsGyTElJCC~Ksw@dvc77D#Y)_f%Zud^;Ws%ESI;#!6$TshQl{Qk8wVtU^@P>t1Q z3(O9EVBqO+t!Z$=tZFv9+b@ed>>WKe;q)S{#@9jjk7rC*Yz?EqcaK+{3`V16n9LbG zWQ2wH$xhWQ%Rg{xQongb5Y)IiNb*UN-gc^*_sQedvvWZ5;3e>h57^)6?Em=?=1cX{ zYqq``ayRwbSYBxnU0bd|jn(wqA>)Gb=z6Vr$n3N|dR@TXUfe=VC75*9otYEP+GDd* z>0;oaq$Uyc%^G{qL`Nc9#`YFH)Z?j`s9)oPwb*2WmsO#vhC)Theg}l z7D`*}jwZ=Z<&0KcPht;X=24qy(psXtAYSk2V^=k#K}e^1Cw-Ddw(jEL*7`1dlV&N% zu1w`KN#4Ns@eF_ZNbYT(0EKZJERI`GYGG-$+q!?3mIRBC3ER+f8`w^#7#Rj_UWHr- z#!73X?@y-??n}*&dfW*(H^S-ztsU_bk580*X?yh{p)lUKd`#}_{`8>xdsN{KxoCE4 zO_vbrkRuBTE?`?ujF=%q^dk{y?#%8|687Zw$N^JnKhqRw+1SUc@wPJcv>Wg{D<}`E zmsHQH9NBG7OD}mNFQ0TSs^e!#84*u*$v8hoBWUsiJ^NgwPL!mSouwUEq(J!0{V5M<%*#r+6Lq_ab)&R-z5e>kx zzs;b#)!=g)@FlSy+MU76teL190G)mKm+SBe{#3{mS@$bs@50d^YGOX)Q}a;N>EZQ( zp{8fx1FYaM ztnN3;07mOj*ji&9-e(3$OaXx6(Ecu|fOs?KcLO!=RRVKy0_Uf1rD7npCj9kWR7>&B z9KAfPP3&HUJ3c)Cp4m4bKx3GTZ_uLacWv^l% zO0q4^Q@HAuM)PHC6& zv-BII^@YTOX5O==jRz_4WA@*O$v=k?F|vs`{BgmPli!PgG4O`yl-LJr${^E+U{7dP zi+l^#Q31wTV(72#n<$o|bQj@_z=~L<%=8uz!!eWuJ?bXN8 z%GWdHrtJ7@?%{lqKj8HahU$#_5`}0>L*zSR4$V^#g-o)g{*-{+HYCyqq(CPaS(VOM zu0wC#P9NCBihvx6dnfA|S6DVsuG>3V|3tg*DfW)t0R&!)aftRIX>7F(CPUw>3V}9&51bt~ zqI|aj;MZPGCEP3}w<5Uv5wR5Z3c>Bbq++@^Nt`g$YJSb0< zcw(75DL=yA-xxywx>3CHB-5{Scz{kGPr%vtK_zia7dZy!AUxGRX;^V>+#MB{Pr;{y zE%Ei?Z0tXyGZ9ER0}I3xoST;b4WUzg>uVp_o{DTcQ}WSSK44(Sgzl^T(1J2h+x*-4 zM}qr*{=wgltrl*6a24JJbWI`D=@OZfzCQcZsq^tV<4rjK@_;2_jNDsukaksMI=#=bLzz?wg}MNg1Ru!hPI!|)du1}J}9&J<^$$T*$GO*_OntjqmHHS zFDQLjP_%_g;=sAv0lpx*&8Q~w>KNcZdOit%P#Y%Wr6;g5R@)K50|zXk%fTF25``)6Ug^urfbFK>g7_E?iso&qhZ}2wUy?gG4I!r1Zb22gyi?uUz(C2T z46_D%T=G~6peG-$Lg4S|03ZlmM5b_n)&v7yP}aWA0T&XG^!SKEkIfY?&0%yA8?Gwz z28Z~G97GkcDC5JC1AAOE>rGe?WXA^s6bVGV)prNaTBVg55+?gsd;qs5Z47P0Byw1P(w1XAKWE*K%->4 zFQqT%U>rr>CHA6vO&)wE$*;_5h~xp6yKI|vcgX;*WhjFtAAQ4uMSz2>aE|32qEmL@ zDliBuIuxw@&Q6n7>}pDtZW#MCgzBypqt~EF^#$seZ{Ig)aXA6O+5STfxThl162@m( zd8fe>bE9JU`nqO(me-AT=@&~-v%9(UKC?{%2wv)@FW(KI9tvt75jI5n@w>+POS+M_ z4lhbevG|KqqNOtry@Rq=;XDUuRN(N zpmGF5jRUwW<|bk_bdkaN!42oYBcz095-9^(>S^iAup0EBhEqZj>`TTkd;^MIg)o)P z)S}y0n+eZ;1MVGY}8u#v2;Z&~4;#`!Icb zf(2?-_-Cs;G=`IhUZ%ZULl-@*ub@1U}E}kVmM5**;ybc$-X)n1f}1d3_-I>-^_V_Df^8ir34E zAKE_5<15$z)s8+W(PgjKQ1w67NN=AwF%5~yC-mai9z6ux!K{D%A$0MCF#5%t#xa-- z!?jAJa>sw_r83OA`jPx7x%e5~!ys*d_YH_T`A}kfH|?Mp>&Lhkn?@+_r$q=jA1UQt zTAS+VC}6{ng5~y7MZMZ>=_!|5qPZ^VO$bv}UYa#RwKqJmKAJ4ptAx7)=?Q=o`4rut zu7Z&}-M}THUu`IkPtVWK!0T&eheI}MR}5Fm+lq{nFYnrOu=ww7Dj#sX{*kGX*`;^X zuD*M|OYhKuRVN7a4ydv2(L0ia-r>ifAa=s)Zv0`XW!W{LcL+WQddJsKdb2?97=CjF zDRi=bL}U{rb?cq9FyQpbKGZ3bFks2M1f6j#b+^oCU2MA>&f3}w$IJSHQ#z@|e%h0* zIO|Yo*5{Uq8n|rQ+l$jdJmk@d9CNv~Aj#mmP-p+?{o0jb_a*7L2Z>UTczMkjdAF-o`1m`)%f$!kT^0a2 zo+w;PZfZ)p@UgP6%G6m`&&IyOLsVYydg{S)X%%Btf)%GeKnKFGoPDqX~DzbFp5Mu~#PYzNr`OZsnk z2upbBT`%&p(!^c?jy0vwMH&TJ!x^0WiJ-xk1ov?}4{0=wsjnYFqk9t;*No+3vsyE? zkA-CX613IpmhME=KP<8NxbWsN^xvSM%(L(@%tHkB{_3CG;0nEHfoHLBpeQIZ$u*#5 zw{b*OuF>|KBd?04-rW?oD< zgpEOZ(L=)R8C8I4>jMT;@%n|=TrVM$Q2CnI3YL;T?;5_k&#{>^A zC6$IlVyrZ zLd$saFN>sE+Ge zl4TB80&;Sdu)!sdafJ|vM3fv1QX4v(WOkDS<>__dI`Zf38ZfEs zqMOpBz3BZ{;aAx{ae;(d9{?MQQ0=M`?CZ9*Ttcg$aM9gOoN1!m5?$Gw(_M%hyxD^e zK;YaKz&CcT>n9X5@_)`nS>^kxrvwH) zxM{{E@Mo@KAj)W80ffX8&-IxM778*JOBP46?$~um0Ov#QwJqh-Z~o_xuM9p_qgK=kx?2zXQ| zyKA{37wfMf|9j27%+JlUnMMro${%hywX`p&mR@p-41*t8qzTD)C$!faq#TOYv(;CqQ7%$0}c1_zCP)C7A zcJ;6mo6L*|Bhm>0uiB>=V1{vCZLg}bv!U9-G{hhs zx38b51RnHTe`I&R^z^wsByBFJwsBfxyw}v(#n+>w*WhxIJF*Fx$%l4waWi8qVjWYD z?*=kTFO0wabg(vMcsSv>b+&ez=WBSPlTD$JD0p+H;M}1 zeChZOFo97X7%)c-Pw+g2@J>$Lh6vthit?t6Gu%HSnOg-{Ja6hg__R)zoVD2}FpDBX zWhH5rc$Vxm0GyuKoS9t6eU&E$=@fO?A@P+^KwFBN*_985ZkNxIsJ77H$99Ma&ROz zW8IhrUD#==tH-$urJX%^cco!nSb3c=>-g2JbQ*toI& zVjnNF2MNka&5ZF0h0+v;v#^ozXVCIQExs5(+z%*C%IO`x9ueANqrOE$0GxnVpW zlV9Gr1MLsmL+C=papFor31~~fzcB;L#$_+TU8Jy|;65UI;PgBOf?Xg~I|e@gV4F;B z;;=wPOrCl7c6%XdNuXD(x~M=cHykN9);c(?yS4Awe9}LMli`nH4q;>HCc+jN{|d4w zY0;(X+1%uImn&XRcG|hD9qCG{mtmK(occqr?Wt-;_fdZ?CZ24^7f@?aXI$(!Qq~dkUO{(?xmGM9U`sFu*ND44{XO>M zz6JL)1UUOCC0qiaU;HhJv&QYmN2zApq-9r&7xp_yLaU?z|4k-Kb6Ot&&{oV6 zi^sR7Uh?+Rgi*53SP;!auXS%4hDo2fTI&?p0+VilO7mxHqIewVr?W;QURFJ|T?!vy zKM6eww8)>tOa2%Qu9Ih^Q*7o6ru*FaT%&F}$Fyi{I#ykWXfqG0XlKtoEAfVDQ*${w zi0*LlsfYvd;`3YlhdtTLo(vu5aH}Oqy^itJt~yZ|Te3heO$Aw?TZZwTS3^VJVTAN& z$4Qs%AJaQ{h%)pNJi&Xqv?i zD$5}&#jbjRpR+?Yqn<)in_HgZEk`=M=SBiYE?p3}h3A4Sz*%+ix^0E@h56IL)N45T zR;dE-TEEz^`g!nrmW!1$I0fL6=xFb3%o~qEk2@M5F|P=Z91(z|q(VC@QjsP5+L#OF7xzEP|~ z&ayOi5jOndb&SZ>_?LR;KwDy~XHheEpc;qI_(k`2AKaEFVyeoE-71gh>RUWFKk3*9 zHLXYMxt@fxP$zlPlYH=H*Zbu4W-3i4%CT;HH9=9uYy997*)z~g^S`VQuM=fqt-Y8La#9-xoea=piZYIsTn zPd`6!lgT51`KsU>h0Te?%lJHMuY&UYXX*=hd1p6pcf=CNxe}L&1%mzbs7IN3C*RJK z2sTh^NLj?&Lg4XEae;-2?{lBD=G1Lf_fvbjB#NUx3jI@_&$UQpxpKTIN5?LW<;Yb)xo^BplI-2bS1ksgQT+Eo$Nn9ugt)JuS%u7?5MbF09*A z3BDJaqCC%X3n%l)V$Z7~t3g>_GJ><`{iFr4u9lNZ5S(GQSKdWW=j z@c9718d+J1lCnF34-a}LG7vQ0MO|L{!81YW;E!(8mmsjvT(h=?+vXSrc)CHhJ}3j< z7{XR^V^-@Mgc3noQ;_G*gPy@GaW#nqaAmy#z7XsoE)q@In@MWeUkJ>5__%(@Q}9>O z#Cp@gXa1rA9_okEf-BD@;-hicxHVO}h`lV?oS9|(aO63`FYPB}?}uH`W=(OP7^^)= zu1oC|rJq14jkG`tWh^bF`MClD) zUJeZhwdEFuh~8^GzQA;dyn%C|#ldYL_XpV@Z@SJ=ZF6;H7HDUMX=-#k&iVYSfe=Ns zqMy8?VOVJmhbAuQqHO%ym0&Uk%7eg)D!e@q{sMeu@U`V*wCsHXa2EpH(x+h_qyege zVgnI46zcH5-*>dNUUL0wXrE4Pbk6B%!>{YF7z3-i9Qye3xrT`+d6cn(U;-Qurzz%w z{Q`!~kZ1p+bHFPmylsx?hQT~Cc-4YTs8Vf zesh9JI26k85HTICDlvYn(ZXj?ml}vqiSstTe(JS`@eibU6ixo|8m}n91bP%?qsLP) zm4~_O3ePCu%yIjAxEw!Id4pvQl1-h551wkN5>64;m>QKd>twdbAb;&vA|% zaoF}@~u@z57__}2&EeAnI*H77XFa^kPCh-{5cGlVoj zjq4ix%@@Z|t;ig{(x=!)?1R%V(l4HqwJBj!v%u%Wn(^j<8GjM63Xnwp_<5+khn*mE zW`n1bg70m8j-?9M4DvryjsrMBI>C+J%7EpDksOxfWgv)C{4*Vdy+%j=iAw-KeL&&- zzka73%yjgk@Q114gttMQj&-`1v`u{et0J1Mv z345Rq!hcJ@z8WBZ{XRkrd07yqr#!qaX!x+ZX94=qRzb<;e0u`U7d3cgolhFdVr1>nq&{w~QjUjF#|}jd zjAmP#oLy`N0e|4XK@cp&WLUudt^-I7?L!3Y=UD{M2J%qwUvD5cpgagz9R5e(cvOvr z&q<~NJKg1m(jks$yuC+bCiX!4cRLJCV~xMQ6s(kl13}?ockp1q;^8aKdJXtcFW^Ik zdJ&a+b7*40SWx?QD#0N|bq;zA3W4%MI~r=1G?!Q~XF+A@0pIYz`Cz;sPA1!*VOIX= zIFm;lx^-VR1^u0>oBE;@$0A^h6An_d-*{d9&!`1(vjj)~YYGRPAi|FDI49s^7qD31 z5O4&;;+56jKY0E=j%fLTNF6fq0*^p>t?>WZ0JtI{*d1d)$DX4O5rYh_4&pjYe)8i) zS032vAm4mnltJ=@g5@g?GCMStt}?|(;Ha>}=%}-5Ka^g+N^$;gdq4cNA1lY+S^6J7 zl5G?*NiwKz31P;bI8Q5w9SU=SvODhI?3xNoG03lj)&KJ5m50!c&`%ToygWl>6PLf# z-lMIWH><)4VTq_DQzVUF-~+sX;Weg`g7BgQz5;y{yz9nPc&uTgkV$rbLWE2?a6{%k zJjXF?h$VomoP5XlX8)!CmIL4crgq{ykV!q@QUK*{98Bu&V5?GP7Sg>lN{V)s0wz@= zn&Tb^OWk4Sp91YP%^^}47hd=Zq)+f~&_;lja`TAg)_D*i9J4-gAT~zPN+W=vPo2gPHuXV6){Zv-7X=@LIpt zp7|u?=vgYYu=8P6k+H}A=#?3&H*~?|4{i^AZt(s{x?ym>s}KPK)$-YjK^oQ{oZ0f%M z>(Z@?zSqtbhj*6pGc=fP7o7Q4M=!G7g?A(1ta6=ihlHSU?A2A0<}MlM*N2X2{MdQs z{b6;<&CO@>+^>VxA69XvomV?9`Wl(G@uVs(ZWB%W56I!4^1UvlJDtYk+uNWm95uW} z)5$JoR=h*?j$Qn>{6fQajuRO`OMH$JO~)h`@9YR{O>O;5{q>U~#g}!%!iQA)GwCyL zpA9|nsL_pvP3;w_J8K#bZ-+0Iy-ZG1P~C9L5KIS={Ng^@B+dJwV1f6_!Y%+OuHmO1 zPp4~;Jfh)93H_Np$FhfmCV-lAeT=oV-BE$;JZVsc6@?QVAF~^p`Q=a!L;BXa9%w13 z(x0CXe%Z6X^fNxrOBUBJ4Tj#iR%g2B%st5(1}$LK>E zp?+8Rt|LikztTj^g|if-`SOYd=@*{IiP*tzC}PkiStkD$AS~-`lm`_>hYk-oPB(2| zy3cb)3UE2_UVk}Go-+mvi)SYmR*;Cl)IiB=v*iI(C&#C2@{Z8>dZLxI>#Qeh-RBqY zrXt2+8l802^uesluXR4InTYwFLc6?B7A7S+ z@kWbvsiahQ)NeP@FzLlg}T6Mxe=KtR^uR)2qzsCSz zC`LEi_1{7P5L&=yz`=6<5&b?`B^EdyyZDid^b_$5+z$5P7f$sG|(Vz?MQn5hQ1bWfi2V`RU&izrkk(i{; zucxoS$hW@V4o?n|9V=KKLvnBYs!{=sm&hLu2~)6KWTLQ2L`seAsc@Q?OrmA37Kdj( zW0F(8B}xk!l=4Hs0D*=a3vu1W(0fQ6Rs_#)tpYmKG}^kf2P~`#+CI3v@8(YwDB4r@Mtt!+gQ)rY@?36pLxU+T<8u4$oj1i`V3Gu!DW9Basv zHd#6}8fA}l{PMnFo`%`>P}>93pr5SZY)h?1mgPdYU&?@7W332D{ZIj)Z-Aw|Bf>w# zKe2^cFi7nbxfM6rbgc7JJHZH?Y%+atP@u*~M#ER0Rw*Kpn)EalL8HHHsZUmqO#-Ag zU+h_uzFU$Ay6U%RL|YF)zw+(H**d^4^x%S(ztJgAA z7Yo5?tpG9^XRZRk2o!*r&H+3EWeJVHz&4TO@TD;{Hw8MW?gB`v+o7~wy=yC&PdI6m zJ7D5wvYCR%L57Fj^dmw%PFmRp9Tv98!{0Kf5ob*w0fPLeT~vZw#0EApcc)j_mYZnI z^1$TqBIpNLY;?eOGUDphUG3(46Qp&Cy-@MZgZ{&LQ2%J+3i}>N7Vuqu2t)yW0FR4t zt2%DIs@O!ta~^;q7wg%$SKp&~d2)b7C`j#9!x=JQ#Sq_nAmMn9mHoFH|4~*U;IQ24>elYKky3YA*o+zD()u9(Bp>zY9lZ1{<$d)U5F0e!y9$ zNzd}78#yYi^j7D*2KFt(ipyhf=UYlUK@5>xUgpX6#1#bUk}qz7lf-3X#Leun*XCjd zM$Bj1eHtnG&LC018fwJ>O=w@1gPuYXHFkC#vbZauaQE=d^@)iSHpqKD@?%Cq@KO)? z3Z(rcKv7g$4kj7NZ$Q+jaCe0Y|I9W3Y8GopbGU!zj?_W=%?|3@7l4hB3VkhFKY&0z z71TT(sB8BwcEF^K(gUZaj$M4fl50t2)NpCXuyNwHa|X?kc`?G<7lsB^VOwB~Gm|e} z(ZZreR2$pxSJ=bO=}pQ^oP$em*i49{Jnn@psC3?{^Zs^%%Wj}(_|A6_P~0ru^sygz z2ic}8DB&lIQV44`ic4J5T>$|?TN<+^m68sMj!TV-(b4{+mj{ZHlIz4iLrbj3Osdx)EBV28q8imohYrh;1bXC;q$%DHJdV>z>0M zhTgB|AymuNnww`$S+lt1t7lO!7T5kudP#OnP$s-ugK>SEBw(dz=;6+G*Dsx=h12$4 z)ngqtSl(LcMsqu`pW?uN3M=leZ+8I2+hr}CEnMNFvFu#Y$E^ju&Od$ele?!s;OyVG;xo{M*qxr1ZZR~FNVJCRzb@VWQpv_XiYl~>rM`RXS&6O4u z$6+ZL1Y54&hpljq-ZLsZtz-todVs3Rnt9ODPKg~VC*!Zi8el;-@4oV(%tjSSb6-(2 z$#}o;k`DG1Y=GrBpKH|_LDGborWW)(%8;D;a|@)`;>nI>p+DPUNnzAMQGk42n`sEQ z0>B)Q8#JkJ5+;FgxVOHW;Y+udh})vESJ)P6mXHEvq@;9&`#c%Jy^24%EZK5tjgW4eI^m@G<+k`)Xfxx&2d5h(E5_JIJc z@bArBmTywrv`vUlKm7)UYG}AdtHeihGgWA+t_Etz28DhPBy@!**6j7pQG1!m$WMXr z>Q-B}8^^7-RMlo^J^8wpI~gAyW_i$h%G$i7d&$jp{TFQ~@JUAmms#j=wsBOSea132 zdWue8J&D*~j37uO0u|c_Pn+&oZ1VeEDRjoM>cl}J(_FK`yg5Ee&}rRHKq}yYDrSob zIT6=D!J?agglon2=jujSgNuEjS-Db|21?-c@U zP=DaU9l(W+Z)&%3vP6U{>(gIn{6Ns))sr}PmoRQ144%S@XY(H2Wi}~@o64Cc^BE& zGFvQ{dKJzUC`B=I8A(@IW47Y9!d*Khlx8KwxjaEYrc(Sy@LiQ?V=qryUy6il`^`L3 z(cf#q@|em%u`O_DhB+(@mD;WeC3is|rK0tc1nEgTY0LI8+_Lv}d<=y&r*#5y6WqN-{%UlXJ z+#td>=j=-*#M~gNf^@7?iPWr`%M{+ejE4kww}N6nDbhA+2Q zLE1EJT3(OKAg(Xz)BvqYhK1Fp`XfiUs?zl(0tsH|k69^{ZhG8}2dl!IH^|jQRoC*H zLX4Qz1UD8HQ?6Vl;0aq#xW85ZN?@pEkU=mZ=Tplo)LYK$=q;MJ(J5k=W2SSDLQ5SC z2QznwACv8y?gglloX{=j2fG{(DlQ6GMyaWMIhRpH^fk5q)05NBmnIHNZxv{8g>aJf zu?O4(FG=&d@HSw)a?@y!ylLb3rsd@N)J*kJg}%T$!8GK!te(_AnI7bn$$}WXrh&7Os&Nf7~>)b`pD)IYqrixFl)sj?sS_2Cx|Ji3B zuS3lE%`}_ydMA(7I)3bv^R@p_6n8{W#rfgmYw4I`#o3NuYV^U87fyc_4#Ffs=}vo3 zaLq+x)9cmp!UOWxZ@9>UKI2!1e^^yDj3Nei&VwxB^Gn9GQovFqJ0+_5TA!ljSG!WP z-+@5Fz`G>wSf+J8JMFXJlX$Xz>ct%BvMO6H$$-?x?iWGM(AZSZ*Z z=<{hCoR~hEtG0!d<6o3mRO-lWoMO4G<;^NND%(^XXfn+77{~VD06d_6&M2s|++XmaCskrzg5e ze6X_=SMGAl7ZeVho}@P{o2>7w(f6=6tMVhkCY79F!Aq$rmMwwgg3R`-v}y0rUYqq$=mdSD1K%V!!CA#U{U*KOIy2p;OEn`D-?(GDvq^!GO;T|hPMRO$+4Srt-9b0UAG<(d(#Or9+l@1Rn1CKM8q_lF9KwDmy({f5oJPzDkvKElmC#G!87v5>hH{Pdub z@GjUw#TivQIB&2b=$AHEE$P{O8=m$j?wC&l>KXgh#;f59a_(vVFhge=f}b8(sq{%l|LB-UQ%X)N~!Pfa}Kk{|&-} zD`+8;A3}RL;r~hh+TT7r60H~_yz~En*06WUB;Pk^D*k2A{GXPg(?Zb~G;Vfjm2mR? zlopC#0TM(!zrit-7Rkfi&x@cY+keA^@TI<9oowreK(Ciw+Ya#VPaV1+`F{lE8d42B z{6=pdgL+E9Ek_r%XZi>ceGYZ{B&UB0%ruuT$Bo#4z#u|W<|wN7Cq4}49ve7`sh)K^gJ7LCM=toPQ9C~oRtLHn!mT>eE(jzad2jbew-04B z{%}rY(r{L(1rw=;i_OkxheA_EV zKg2@hLug?91k}m)A1*)xB5?leIY3Ksu``LU!^Zd;!CIx~1Z0RgxE_gmxI3eMmf(|b zQ0s$^i3Q-RFcPa8DjOn3t)ssIqoEK~qw@a!K;p~b*Le=I`9khAclndhimormzBhN{ zh|hi58T8ShF0K6!|AUnxiy14zNcKbXVj;aLAXsVxMj<`FC5-RYWgPrxVo0Z0*w@M9 z_wvSUCeS$-P0HbwbeCRqw#M3V}jt*fXqwb2o4e!EV_Z(u(=5%R3P8?H>+1Mt7{ zp~rMvNOJoX_Y2q-{}69k{Y8@5b#kLv=9-u8%UfRyx{dCT1p`-zj2Dmbn%_r=L8zy8 zT}Qc(@$8}BW7$XO?Row39cn>wIns6z{}n+Cb31kp+paVxPeqpKZY!%gmg`P3^0_?!MlSeT^GF~~a3Qp(oU4!u@U$j8*#@B$3R$T)hO5By*$)E$ zX&F*HGC>p@g#!FXp&=H3Pd@h6Z{-$_J%i^T+los&zI%|`{J@h3Zg`F%A%KLGyA61V zT(N`p`X44tGMR#-S30X!z94p~dNFalm)rb6hkBVj?xZwCIK%E<%gEUPrfB+7z|XL_ z0iIi0FerZV;gzQsX>WnFJ~b)9s?Ge2z_4`_RZw`&o333M8#3aY2BY%dEfYNi@Fbop zKR!Tn)EvoM3waLk)A8`?{*SXEz-B<%$>$3;18#b2AKZpQ6?ULfBr$wB3Cd}UOB8QV zztplPFY-I(r(nO+x#e51hl6-Vi0b8CE?|8=hBrBiu4J_DSpp)0^x`hO=CKwOYizji z;#!~nK)4>Ix%<1$h*FKGo$|P8n2A+Or&09HaSsvk*`-ES;mq@AJ| zi9+Sy-XFD-*gw*fgTei@@AUcZPJvG$cuZs}*&IWF35*z|G_Q;%8EQOiiCcKH#KP1z zN?qnBTqgJqbw#u>0&K#9jW6Ca9pWY1wbN})K3Yg4!;80nYH9y7eC2N}2;f9$LPx?! zMsy3G1i@C~8W}MUx<^$p61=tPEzfRMTVf_H@>Ae6xd<6Ak#BN4r5;ChZ&?IU-C}(E zGrvTN!C(IE{$E9C>)gNbsQ&x%Af~=yp;6=$9rFSLOh1IO_q4TrYecc`ID=f2>nR{4 z3V<>+mH0bGmTLc`BLSU2y_q6@V&1XiVA4QP)$yGcYAoH!c( zL!sSb<|_=EDD5(ZYR$V)fFC+`SWwIdY|I?ooi!LPx`Cyjx>&XO z*2(enkC2fqhk|(82WE{Dy|p-Qlks`zl;f@~wvp$6si}ICJqevqs-hJBBMkzg*bsiz z=6mv7hoW!|?ohAjQXHeiBF?e!kdT4Bs&M&Eo4V+Svl$6fVDuJ@%@P=+aVA!h1Rb$5 z)Q?nNZcT&qM*{dI9lcACTiaG858$zn7PyRa#2=JFei8!XZzFb{WU&uxpO2D^Brb>b z8v#aQ-%NU?3(TZ_AG-IUUCP{kwO0Hofk$iH2$Cl(-dW*N9NP!XUD5k%P5e#B`d^y^ z|ArPoHFjC-T}lMZ{Dptae2~IHR2LGmry_f6V`}n)Gx>?q6n%Yo7%MXWm-)>v|&m)9B#V*<(G54;h|Br{nMTFIF z^Dr{s&Ozo*_w+(Z0E#>dl{*a7De@TiHymiD-Q>1TAzd~2lC$O9@4} zSndO2LGEsTdrh&Jl2T=WBEChC|@NL%I2UWOnO7aa!tWy(bV+I6zXn z0FwO#k6+Kl$lC1hr92?AD+f*NS`kQLU;oX<`5@?Zg?(=y;HsYjl)~K88d;szjNe`F z6B#tlLT#v#L$3h2$#^Hn6wW_-xfiOO`MF;TgxG!2*x=O+zVWsCL#T*Dm<|-P3tnVX ztqTAk*!9ry)vn+F2vYLi9uJUmYD$8UzlSb3226wZ*hmD4T`id>Is7p3@TV?BFrC(| zgtTXCmIfpZb`6Kn*RTK}OS;NkM^@k>S_r{Tj~XMu!h;P$_VjY^;F6JIH*YtXhR2|a!FWRmtYl@V;as= zgs|@dLJ0o?gs`2#*3YRx04}wSAQt3^d%)nev5*gT1&Nyg^~rBAMQOJ; z3@A-N&KqP`WL(vga`e`a#T>X2FF{8{wd${+&DKS=q0oR3gem(0R^&^72y0&kc1P-i zThd${3g(d6_u^tMEZp_yg-b{y4~Zn0rFs{~kVPBz6E#D{sK zkM_gCh86-Y2z|X)?!C2IbZ!8gSmlAuYy*iillm zUnB?`@9*s^bjY@@m2k5(jGxOSkXO+Zr(|mcY^#<}P4-a`!(#5s<|2+m{5OG3Ux-Qp zDlf(4!|;NRwujbFm=lwH@EhX+IHo>9!x?;9dPyLN^~2Sp7+G`Y2A*62^feSo!h6^h z!K!f<#6&HPB+zDw`2T6{+QXsT+Wr`mJwy$%<&b8aCX}3_D2*YFLur?YkQg#jO3tDg z)OKztG*eCy+D=6jLs5f~p_~edBBC5Bhf1aIe)g{S-S2n3*LPjtAK!a@{u*;V^E~TW zYu#(z>v!M3-)-X@{POG@;MrSo6o@rM$~I6r@jNaz)~dxM4g4TVWNiZ%>!*-Wx$r7Q z>h3M0s{%lyd<+x$b?y+&`~7gE=iN5#W_kC8YYh(zd=y*S6B|-yR;)3!ZINp=+L^FA z2OP-~<4-RHl_J|ASfiWui`={kMw?0Z|FKex{)2rCN(`@gmqvRE74+{tc$mf4Lio!- z7ggJ~amV~b7g4wR;b#cI6(P)5RhvwoP}dVC2u&E~XsOo8VBjAI&2*UVc?07g_~_Ic z*C=D3?mtgmh+~Z?*L#D+XiLEIE(!_rb0rn|+Y(nUJbu#+5lX56;+8wG!%FwQL`cU# zKEBjr8tQi_)&E`$9D-!DqX-1ikFGXsWFHg*sliI0zRrOsa1gC{C0oS0p>_z$wN`+> z%;g>mqVrw@OGLuvhXE_$$!K^oB9c>cV<(1Obq^AHmZ1?o^kC}%sWf;ztw|y zDDsGi>yhap!eQJ3bfpOpm1M`skyUWX* zo)2GZH!eFq#a%2VoM=%=zqH@@K+P4IIjv*|*+iAcLqGxb^y{zAsnLDJywO#`_Gy;kwJiBFw_GAy2w zo6X-}z_{%~a!A@F7eBlWwk>)G+t>0VO-AZgq(_O`No!lP>(H3in_+X~hdbxYgF~ds z;gB*GTIb-LdaWZ+Z5{+^GSbpr9H`S;c;1tP~d>8Rw7&v8f z>b%RY(WVhp0At<#lqmpFi4a6m#@YYL-Df?1&HEJijl0#vTlz5nW_hrytU$6k_80De*JS?6$V zhV+G~0N=z#!0@F4ry6V70OLLnUxIUa;f*M*K-3d*GrzilItpn6Lrz^K2Mu}v2mBbS zz=X_PMCUXBz@f!5{e2ZsGNKX02-6~kSj2TMzZ#)0ZH*r?z^=ep7dM6_-SD-lvZ#-c zs<&KcJ)8k5kE|C*%c4k(JMTm!E}z??XE1Og3ZY4-4hr|0JuR}g0a-6g=aA7<%a7%~ zR?ls$KQ%t9z9}u{;vM^qrgc(=9M@XzY`qotlP_-#g>e3+2elpDD>w$Rn7tr-cdY+h zU?USzwv7_=>Fpxry#2}_mT&c?!hyZe{S>7eij`1YnwMloH>g6pZnzcf9{ii7t(?X zT0b>95MS$u>Xt3!p7M1c>1_%F7Vq%?oC|ft^gk-iyM=&3D0utJT zupjXD5gahiknMbH5%^nwW1Ok4LW90hdME%}Js>C9gXB~|&~ZPom`s5g1$>Q*yMKW4 zj_dzbR6&HO1#v7T7o^zCJh%ZInOayKl9#hU5VHu_Q_djhdH^YeGhGpaWUeCn{(jZe z7=Y1#D=@LU|Me5!8V!bVuLi+z^g=g1yG6^zYvLXfs35hIT?3Lk5Yf*E%AQwd>vtH0 za?okwwO<>!30V8E9?$QxLRNZWJ#fc+fVbU^6f!dOAJ2x;)rgMe5zP#PneVrQ@^eDF zo#f9Soa(WA*6-*TZzOVjU19beeCel#Wm4BIkKMF;8}|hrvBzYY!HJ?fZq6d=agpm3 z7k8}sm?*tt*^Z;qjv6JEB~lp%uTSW1edeDz*s}FP-aW>%Bbn3`iwI)f>odAU)^}D; zPL4}s?KJ|mHYe}uE;=egNmMCJS_cGWk`^Wbz`*AI9$pnG&_`USS(HkzO|6=5dkB2R@#RE5eu)OiiZxO-Re+Ijr|OvV?Ow|N0T-W zW7`n?>sG?sJKSl&^XgB718>ev<;x@JgePPpjY0Cs$@I!_OBq@eU$fv*=8&E>)gIhvwEx`0HymJs-yFLKo<`sZ$#2CCi<#w0kT90v?m%F(* zp*R}~m08VAn+@^tqE?}@m_I~hK1U6w9Xzh>kj8>oh11zq7`atRST?A4G(v@&1|(U1 z47{eNZ{@2&QQz9;P}tO_3ZP|9RAbkK%*_EK_id1xhMVb|^buHU43Ny1OwJi@BdUSrTvs<@(|m>wJH{-;am5207!n8W(Y4MZ|&SYJ<&@XF`~X%EjMMsv*~%tUYY z#Y)}Pj)8-JCYBiPR91SCQPL~JACydp`(5lVL3ovH4hi+FFvHL?%i2nxz5U`2>~Z`WW*Emq!_h4d0sEz$s78 z; z6k#$^HkdbYn8}B{{3x!6t#=j+>Zmr|3K$W#71!}Ia9E~N#F*q$p#EBMoP-G!2bY%; zEh$l#)Bzm1s2ZU902?_qVkCvz3?@UHoQx|qW^W6ts)jI9rK7N3)p@opa{!K+CAT|g zUC8EK=0WYA{n<(;(Ywn+jWkCzFy4e86I!N&!&IyJZG7V93f|-D&b!`-DFy|}{o)S7 z!S3OsjM@cK6DI~K4fEg%fceup6$FBNlz$s6DBeC=Wf5>}O7#SH4d|3{Dso|6`btV) zhI^Y$#{fPufK!}rM2XRa3YF@Sl6bqr3K0O5vYK9Oe*{|Q<743&6Q_T#$75d@2$@uW zRlu?in~U;3G~AbpBEGQV-Kuh0l7-Z%XZ)|9XXkIW!QuPjy#zngKNB0Yki4sDwOKHP zQ4go*b8C_e@k=AEhf67w zj)a$A6au=Xk3P0rv%XMQ9d`n-ivwnhxD^#Rrp8jAVB?zFaYnA^* zr#>D|?}cNC4&|}d%N|=QVnq6euI<}g2U{eY_}TaFXHSaY~#1O)0E%5_02^l{meiHFna}eI!hWW27Cny)yY_Yc^sa2P$oK zF04ALr?)R!<(vvf9KBy*gF{EQKwY_eDQ4)PN$$a~t);aRuN8mCj5y|WJh|{9IYNqA^I^WWVUE*ayT9 zmT1o9($e4*WrqN&L4KE$gT%3qvA^XE+Zj(nQipnsGR59b1A$x8Pp`BuSi7*%K3wcR zq-f-p5pnNNLiFS^Ka=SAosbunh-pWRGW>X7=e@u3r9;j}*Ti7`O+s@)@nwJ;l)Rc%?_`#}4 z8>PsxybHU%)f&|>_lZAmg`FYikDKQdz#U9qTYka29{S*A#$OLXNs3Q`b8H?5WmR7$ zKiS^8IyM#YYg!4^b5r$nQFl^joSrbxJ0CK%?CdvN)XyLyO(q;z@bm&@0@xY(_AQe2 zzR`!>qWL%0tY#h-r8o)*U`hC2!H%`ok2>>sE|PxSs@sBsItc<68hDfdmP^Ll-4id$ z(v+1RJZoLt@fa31#Rn-X;AbfGyhKcoFZWGBd~qyO=BMr9=qL<+&qgpyA#y&|i7T@1 ziGYLKigE(njg+nWbH9>dCI7E~Qum0*8?*D{#gDle4$u>>x80ILBadSf@xGDgClHT9 zdqI;LcYIZ-Az9Ubd51C5%EJ;dyLizSCtsD2c?nn6s1R@)7&T{&?)B!bk=2t&cZidC zW=C{nMHz1}kjvsVkbj`(=+QD0{Rda{k2!`Z4k6vpZC1}=I^2t>(+>c!Zqp`~Hvz@? z0|TzDH)PP-SF$rlA?@uLU>*ay=gr_fL(tbh(`Lx3m0sgxzsql+)6bwkj7>lZgvGMn zC+jvw%Nrbb+7Pf$4Z13WRc9_AtI~$h>IX++stF1mA)Gf>z)F5^WLzkQ?!bM5 zC6f+n>|;C!LNJPy?q?~OjQ^tk{557Uhi-!Nf~(Gx3YTQt1( zPW-0rQ{m>GfbChH0sX9b2L>Ak(ifvVZXbE3NM2$Gvt$M-dXsU z$WYhS<#YWzUm0QWS`=u$N#~dg4tt5t<@5N~_XO%CdNp*lxm=8oEnA2I{yEuGQcK4< zy1og=yw<1axwQ1fS82Fu3bo#7Aq`rO~)T7O9VO2?Sl*gt9}i-4x_4oOyY_iaC!Qxay6?SfL1# z#Wb(g7J{0PxrwpTc8LR@*E~sVJ^be{a+IJPyu6A zH-sqLm<$v~QH9Qo?LMI=hYrWVv*Ab*uJuX+jPL|>QoWejEFQ(U0?*soAdJDkz~ZU$ z!p1Py9t-2CjU(?&CZjPa1Ugeq+vDF}i%b|$xf6L4OU!iT zfdwW@fQfl;W{WY5u`ogFma~jk`@!qsHbYxq2we?<8M7ODE%yZmjXMF&Ny22oSL5(0 z=U~i&Nwxs=jZpR}D=f*?3{$(D1<5KjQ#es^#5cu|;mTRKvh5lDS5mg-KkcqYNMfFh zr@~K}C8p2tdlLV$vtwKL$f7NvMancg!({?69BMMyAEre~fgZA`=v2e5V|0Jj)7266 z0b7nok@J@e3*#u~-~;#KzrFU>M;WL&KenzAB)Jj@f&y4GOPFKVUwY|d7V*&8uq-XQ z@>HO$FQBaqxtK99*Ddh+Ns*FR(q9wqG#Ld|NJzZsiG&fq8K$lI<@<1&RbbZ_i_f5p`?EhHe$nkp4JU`q$Yf+sjN)G%fJVTeLt6qf4z3_2o(_t z&jxyJ^Z&`*h5|lM&sksDeDeR(+#Ul+e&3#z+vaelLTFsfMVK;sA9VO&=S==uX6720 zR$#>SFbHS^Tw4?h;|F~xzl?~Z)0ODVlggi>Q5Z6j&J=!5zO@o9hJ$fLWgLXTS5ka; ZrtwvY!Jdx2hfwfmv17McDcLphzW_oxjr#xq literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png new file mode 100644 index 0000000000000000000000000000000000000000..d90853fc1daa6fcae2eaae658b8936ea3ee8a696 GIT binary patch literal 53972 zcmb@t1yq$^_caPfKtj5^B_*Z18>Hk=!T|;8?w0NjC8bMBL69yf0TGdqZV`~~xclfY z-uK@7jqm@Bdl>_Hd}7zyd+j;rT08QQx&kIT89D+20;ZCptQG&;V8!h<(VlE5gK8z64kt}po_}3kir#-Hv1KgO!&;0eJWLeCGTh3LF;*2)8RtE z$koQDCf?I|M9CdN1%^Sn3B-PUq6qUB*FTg7SX{qoOCnuJl5Zm3#S$>-iHSnLTXU~I zfb&| z5{2EP8tBxoFyg6km~%x=VmM>sbvJ!+2+#WZ`<4kb?Nhp_1lBfCRJ^n!j1Mr(3C;II zAdNI@7oK6IG_2}WOixmV(PESOf;sOj1tLQbC>2fB?j}p|`UG32?pqMQWHqu{N!S5p9#LF@9^4I7RwXjM1^{%Y3~Ug zIU~hM8UmG!{2ul%3TPA8yb-kM`ZkB!tesc2KZK2BAVJ~7Oadc!0*T~3S`=J6N59#- zr(NAk6WEWLvVwf>4X-qdEup=R-b-+d{jAo=r}T*ZhGozxk9_Mci*w?QIftW4>5r(( zmH9`MtSiEe;x&E7dDQoJF~pjs(~2StUlCqt%mie1Vr$626x)PoW zhwBzL-Ed1BK*z%mhsk!&rzL)dZy%bHpS)#kz!3?|?o>D3sG*0_=b68O;(rUGOkG}@ zIEFTS2y5x>Umge&ei3j+?p^mqbzS6}(21~r;wj-?iPFMYAfNt2WA^?}ta-dk#6cRC z7)@9Oo9c1uBM6G^oClMWna`&^?u`978yUX6gYfHHXJ=>kLeuBR2q#V5I}2CYZ!J1B z!g&xZGn??7UT1X|5kq`cN;|WQ@5Tg@)`T58LF_v)&VnRy(CMTo6jAVDNBWN0(VaS6 z_@C|+yc>9DV%e)xiZ5lIk}BT!|LtnrSX2DT^SMW@IfsT2;LwA}Lu zDzg|Uje7)sm!xlkBm+gHR8rzY27FvPyr=hhs96+P6WsbO3W=t}AJQ`=xF`}kV|WD~ z(N!mEFo%`r8I;mmN5;vMGUH>)tb9g$eqR{*fwanJ#!a$!D089wc}$-%Hu>I>`vqZ{ zB4~ef1t?MbtE93C8a+q<7JCe%cA``7{?RCA)Jf|pI?`LR z_;$bdOxXACeBk>Py}i4h5;w7uj3hYQs5wadw2=u)8OqFzI(QAZSh(t)tzTt46&Uc` zA|A_nG!iFAPWO=Z^!6P0i1n=Wa4gd=BSgx6p`T7{QA%XMDht$6Hc*+7vt~5Df1DuB z%)&;>Cd7iwQpAjw8kNeL>ds23{h{(m=LfHTwyo3&gF=$m>W|f;)iSBdzG`xz zRT*jNz&QKZY`&M)AueXY%Q5If4m)u>wjF{u3ely{ih92deq)`Sc{5QqQT?WDCqFw+ zOs!e@vef-OR`=+EehVoRx=yn1}*5M6ISQ3(Z$8Q|#V-+04+a<{qBvgPOK{X`DC{GR_<+-1sv2`TLLS z`zZHm@8cy8-_O3EtM*29MJ-zGG-IB(AcOl8&!@+)>R%pbl4r*A=2|_m^#A_CwzHkF zIvBH*4{&jH`{XvegoW2DwF8$)}TZ!mX&=44fj9%vi zcuo7PdF~u}^MA8u57_=z8{eYVDm0(j+U&O}ZY7@S`@v7mSE!Az4g31jm8bvrs~jG? zjOkCtpVDvQZp?0FZhqTh5-EG(1yYyh?v(xL9$1%<1FkIT0=P^i$ z+x>)zr}>rn#j=!f_vIDj^yDNWN%47k*HzgcxNB{XOnErnIH|1c_89i5EPu!Qil>Ed zjX%$X!tj{EoherRRjMnkzJz|lY>Kq9lHzn450|*(Rr$VHSyfq4*(4KGErWk)R`xow z2|r$4E3@aND@|-dT1Eq~22+xDH>sj!iC)R6i~j7yEU~D>yz&#yg-nyqfLBY|nu+`D z<039{77l3+kDoox4P=Bed8Se(wQ^tjsXrY#$?BZ#nHBR=_OkLCmR5ts)%ewoty(2F z%ev=lX`T{{vQ!iwMR)gnKS;bnJ6A(C*i&eeF?(<>caZnFq^5kY`)f$q-C`?MeD$9h z_9A{`MO3N&j}-^8GM(rZ!KBrTyM4Vo+0j|VZ;X%L#OS%V&DOTm zT@9J-E+4kXox2TJ2dnU;xpKG}et?lWy>y&DUTYij(0Q&ypet&?Z7Y%4) z;8yje$a%$j(0X8Nmv71D)wfsG#tkP!A1d7@%daX zT)k7O`LVvfIE=}c#+PasVTk@-boy1z@h69v1?%@e9M~34+?pMIPUA8tcj|ZR>qRi# zIo`S3*77i(p{!{xn{V?!q{)D`#N@}#EEOO8?_(KUfv39yIUl_G5(3@NWfK5$tv2h z(wg>4N&RS64&@Kbt=1#otHT`PeB#_o<=d}kZ!e0Dhk^snuP4TGzb+gqUs>-RFAI1( zG_<>UedovFbG$X4ZyxlNsi?xo46(wjE zx2=Z|OgP++uz2i_P&$C{dGS1k=ZXJKGNG)$MJnxHZp~E;Gn+la*Y4ZXP4P1impg&2 zhD-YxTNOnR964l~kvn-KBi(MBmG?#TlreB~?(oa`s*S4Ly)Et^&64Q3cEKnt=UCi0 zF+p1P4S5CO{|+cc)HA4(g_;@yE4W5QKtd!#Kn7Qc;8z@x{GV%iL}r9Lzwbj35W=kx zkp6l{9sGs=B!FM|H-G-#NeV-_3;u@(em(Obf4>bO%DeOT^&LU*3_(g;Mo9_$)i!s2 z_RQYpxr6J`3X2H1f##^F?}C6p#0dXIRMKMD1Lc3S(t)}{)l`Me9qc$@PaVvjad_G} z!plJr^%Mq|cF$a4G@f?0_AbJnVsyXX5C+%q+njVXzh7~+5u<~uJ))6uaDGO^&%wpP zMJJ9*VQ^?=t$Ubv1ceZkLwQ{hhftL$2b8vGNqoadY^v}-rs!pY6S#raR!;8RifUExPop3iLcWv%SMn1MRP1$iEd{(k@e@#Nnc|Km&O zzhCn4{@0iP@#O#hrLM~}XBh`OP^GK*zvlX@+W&m`*N38<@Sgw2Nc?{i=&Hj z{$n$7bfNPOVPHlID_IR4@E1%n_zz+h_=ow=U--3UjQG;vB?5vZf|9J1jwj-FHrkb5 z_suPna+I=_X42D0e!}UdXA9}JlX5K5gKewwML0=p&C!GIZ)cx2RUK>jGkhoffcxw` zIlt~{ne0fwlVqXc4PS|&tkHAdk&RCqEyaEB-wt17o%^mIzw{c-GRSd#5r?EmGl6uo zaC;fxHPrs2ICI@^B%DDK!jOmdHHwwJs%f2!a58~b#x+Yc2?0$|KCr8+7Tw%ki3TaC z@$yH1*Yaq<&0E&_U~F-c{EQ=I2txt`gn&q?9CJMsNm+$v*Z);l{=4J8Da8&Ya9{l~ zIjpr^QbHHwe@xC9sy(7~H!r4~dyMc_p(gMD^> zG@T@bP&vjQymolZoQNACy`Nt45+>TueMn8eXU>F($4rih9$okPSr5Y_Cy3MOcw(#o zn|cf>4I*BIYG4-~kF-4K(cWD|bPRrEJQ_qPSSS3(BUVsR5#4~NVh0RdVd+u5RfnL_ z>1bX7pPmV!%%iE&#CGixgz`fN8G#6P&5#3@|jpwh{Q=RE%*B@G}C^6x{WEP;k z+;=RU8F1)-+u>s-cO6U%*zFm-+WR#zX-Ha%pkbk@BXr4<381b7)N0UxC`Ry@UFg5t z7m6lJM3ipg-^AKg1M6lrBDk*s28SIq!EAt=0*xV8a`xKwDGc|0z12ZQf)E4ZDe5C= zqn_m_ZnP=9i00hlnwmGH7of}cLE*irL-|EHpu(+81jGc%E6}H9%Mk)48V=B8r$h^3 z&`5D$2x?>v#k?(0kdAR`og_noFL*pqO~0N1+6a1_A?DM-YSR}-8U28_I}rg*Cj%52 z$`&|~EJ=#iGpC`zKhI8&A+H8uz`+L%&5-0J#ZyFtkE9Myv?3%F6ih(ZvjHk=K+T?5PQeK72c% z*5+jAGQ+iGD$upr==6m7rpAkdAopC`QE8eOk-WYLVz*K)v-^83sO~> zj}o_&A+o$uxH;=S>>r)9mFV4PrU=M5dF>y`iWK@2JGXC7^x$wJmzuD8ZA0YnP)qQJ zYe&WpEB4pj>~^CaPSN^E+I^)P{?nI3br!ZP_i8OlM75<~A;UL35?m-PEt%!$hlAPC_0^oO z!72{=v?1I`FEO+?OUOlM*3hBVt?D73mkGJUD}JJ;u=YB_JOY<=&6QrxEMJ%O&a>`M z#$l1fKN1w~+Z{l==~bJ%vCkNMv2-&z5;4D!;Z=&LnIDLqwv)Z3C%*f%nAa^Cjl+A>2hl#T<{^@V-g zPcA!+TW?_UFFl7`xeg&?PAe^su3(2o%;JY$Q;l{9DpV)@go{?fdZD5EVS*!I+L)Si zFQ-Y^vM}sV5aBjePY`3-Qs2wGcRW#0t&3EVyQ1>?k?+NMKkw?x8~NIZfa9zt<0~6m zR#!xq4U=Kqp%~`xXggw~$BI|wG9TZjn55o~zx8}VfKA^LHM(f?(d1z~WIufU_2YdK z(Ssfnf_REG>;ukM3PESxdym>>4^tK@+b8%5-FHYEG;TI}>25%)2_#kywPiQ}%K5hw^mc>KcFdCshSc<2Vt(Cz z#lV=1aa^_`Q;kGY3`A1wV8~PWFzt|`W1XWwMARLQtyddI##e5!jn>wUlC(Go`v!lD zCJ4!BN8qnr!{dlm*U+UtD5d-g?#JPeDG>E8V;e86&@tLn@6R$tw*fb;NCV&cwpnw! zUUYR;6@4Z~OF%#?%t8naHNKNqlUGw?!*V1C0nUwuNfHK|L||qIZyNw72RxVLAlOr< z_yWP+To5m=#bQ@K@z|Z^eouM@CIjtR1}5AGDpLNb6l~yK6KT35C_IO6J$ib|PS2h@ zVotuX({tONaW@>Zds}r=|6$-3-p)WIJhi;6Yx#T(DjpXCBE~HAu@}-9kl>hZGo2T@oa4pY>j^`nv z2Gd;CQLIXgGNQ*04MCQZV-&Yn{r%E5ywSD!HOyL2(+m3Ng)cQIR`H3>yQX#Ne-A4H zT1qAue@Ym7G+l^X151MRGEI67Gr>p|*^lVomdu-l+YD73NEZYnA|G-uP>Ij-;NU5RJS&qB%Q5fI<&Fm_?_sdj;irb%O34~pA;UI zxV?G{4Y)q`yB$hQnCa_ADqd6b582cShz*mW0 zEv9U|=%!hVsB^jauusqPkYN$2s8|VyWnU;aBs8RVh_jvjWZuUCk^GLJ?bE}KJJ>qx z?V5PSH4)PBFk^WCHGkCg0)Rq95dT>p!*DOfmV zPZFbw&Qp=y#)1rfyH`+;Ikz1;jSNoo=n&lqnQ5tJY9Z%B>syjH-KEAPcW+U#1>ClD z#@i!{4pcgBllh6tCBKx8|D38V2rwZN@x~zL_+$vRBKbk-+)qn& zWna^YG7koA8Iwx1v2cmX??}_3y7fdW09)d!M!;cS^w-%+{OQblhhKJ$OJVckm)k3` zKE7Y@|C#MFBJ?DlN~}dbmdr~_)vG{0_Mq)@S44u{ zQ)y*5X0PJ$n}gl9OZ#fQ>L*WretXjzhDq-I-M;zFWVMOJuWoFKPkTccr>GgZxLGPr zFk{p+zHSvw7$tvYpV?(>&5x9sfZ6MJ5e9*cycB%x!oIbb49x5w+XXchNY5i0nc3;g zRdtZ2Ly(0B?qG|4zU4`&kZpO%dx-J%XI1mase^yya*NmD`uyRjL~i{otb2fv-Cwx* zS6lPeRq&6ci+=j}%CU1|JR%$fL;I#8>UL6x_x5umWSNa(?q`L*$=)Bg)KmXgse}8q}Z2~BdvkomYyE2P`G*R-r ziY!fLP2=JVj4O5LH}&5RsC>3-Tgt?aKFj17vE+AsePcK&vfqQ>s;wc$S1{i993n5J zs;4Tb4P8_r`>{m4XjPKE?zS^CF)ENVAV8Km;1wDoce*=wz30qQ=CS!Yyc{>&eP`zU z)UD4H9m8sBOJc$H?QqL$neg&=CsXFww{5eBT|O(Z0z3!Z7#^src&O(+a@??dyw^1~ z^olT8-@qwhR#gY83`?=hAYpZ`Q;trEq@rH)JJKkHJeNW9)EV1gWCDCdz{K|_&?K4Mi&Dmu8&02W!rLHHh z#i!4mU*DnoYyRL_E)7NgVmsSdd%OMS{ro}q5|TEDFa^@x>}I4c)KD&e6{V1_t~iUQ zFj~ydqE}7p`41DEV+@d1LQG$~_3oi?mBlN)&vM=ddb8ebNf=d&Ei zbhS}4{kdw&vA8DT^7-cG)yl!zP;0ST=7p8``D*gbTi>qb&4T!o?t5ep`>_i&+HP-r zZkt%x{m<8kv9;wI!^g7q-&*{j**cr|`Q-(cYx;%HQYbM`&VH8|g#Sh7M~x~>|mDW4=*FhE&(2R`ZP%cYWsf9-9N+jgvPl~^Hym28$D6P zRRn$9UM`FksX+X#<><-ynvxrCKNH1)Sah&-ia&ka6(b`i*s^TszN5-;o6%&JG8fIs zC-$1x;NkY)XU7uWq|xArKNL}!(hxnJb|65T5l2Ga*;TKvu)d3phuJ!)X~ zl!aB>77&qG1v1yvHU;L|E*I+Xlk@R>A}6H9At2%k>2ZZT4$C+}1aAMJ?R-6x%qv9G zqJeYV=(I+Z0N9GZ3(`O#?8ojg0p?7{7cEK@)feC@&?LsxRNp$C!}!Y6)fdM?O6H5{ zRf~IkU~FTAwoV`oKuRJ}EmGg)h>aN}7NlcHdF3gM&o%s;$|_&|azf2KT_l>d@U&gP zoP>j%OpT?0^^#wudig7&nQZTNJQ}!K(v2XUn01b^#IBuw2LnvfaqMTDRLLrWy{iMs zyy$NaM?|70pZKWFWj?e@N^mD49StUaVA!C;$-%~-l{JH7nd4O)^;kW$)FTaPTyQN# zJLKKcqxZmv8qn%Bl^*(%KOG=sT%Pk-(7ky?Ws73*f$JA~XC_&$*E;Dw@HwT!39h22 z^FGxH;}0*A`dDd9?R^Q^;|K^LqbsgOn*z(9s@2(c{e|mlBK#CM)mWEnbubyumTQ=& z>->HxraUC!e*%$$;rgPH3O{%KI-{1sCDpH%dsHL{COYzN=Ku?;Q!k<3+2}zw0r`kL zRFPhD|K{>n#VTzGO~m2>S~T#wMxr-oD}xKilO~*n#*xC4w`+Jr)V_N`3EE79-@h3; zJP>x88M#65-WJr(od71W~3vnL!(`^CBTDD(k-`r{cf&9-rmH? zp(Rm-X(=bgS&?nw9ZE4j`t`JWZU`H#A5&1^72P1ohe>`Mi+L=OZLS54U=&PIcESn! z!4#GoUJ}c}xW^I)@z9ru!OE3+@I}0?qxikSuC==T#z3JgPK)B~JZP-6$?$nw8BXPF z!b0vA_T^*K618#U^d;++;gVec8SL;m3GUFR!Ym)IHZFJG=blGw=*miJM4PfYZG9~x zWYcH4j1|Pd^^h3GjZSb77-r(=j(f7WQ*`Zpd|aY@&^v5tkH`GbEJ*9YlNRVTp2@rC ziL4C8+Lx8mUsSARWp+<#E>3pXRvRV<+`k|I(-}CNsbK zgvg&llr~5Ue=#DIajdkQ#h#G>zB>w~GRbq*%-o^JEeJiGc>bXbMWX z+VnmMRTj@WKiu$HuS)f#y88L~b#V9ue(|4n&v6I;*tChEVu#jrs8%hGWZvGKwofEM zDZ{;b=1IKU(Pt;=i4-NHn6NXQUfu+6csiiRnEg=9%R(BQc1Yn}fb&qj(4Ooitxk_W zB#4O$($EEIBIPm-T<|iBwzgW`wA8wB)?=gIgYe9inel~OtyMLgVyQpa41{&R?1JRl zS+(cQ%jJ*v9EHD2FXc`RIyz7b-Mlj0P_5klY?xx+T_bV*LxW22rD0fR7X-HNwKLcJ zK74{#m{U+XL9rSCS2(r*XQe$w$TRvy>vmvmN-tJfY`X@lae4odIj1Ttt0tKYp$Ei> zsu-WLMfCdpC_-j;XM{;J!HkQz;yKU@?H|KF_V!SPeJ#`O6CO6AN+fQ2_*l)VAFzt~ zy7_c>N`NTaaLz`o#_~fG=ol*`<{5w;9P1Z=@&m|=vayp>?}qiA$Y|_wC8i*wF1qZ8 zP;0z?h7N!FWQ8g=^k_UXhxHeqE*TclflvkqtCdLLLS&18h=9obL1>$n6p;1^Cq79N zDZx2}(Nmby&G|+}%Brv{oI*qcqDRHWi8<+RqC9$5_C5X(0VSbwoG9iTLIJiioR+ zhFm@!`(1ru3h1_RuQn9Wje3F$I=FtAf)8il2!C7Scl+1i+>Z<)6i^XkmDilVNwyF$ zS~CielYNgT^jH~N%%&NE*_|S}4}nfD37L@XnDD3n?ZX}G#~NoOjn=Zi`Bp<~rB!pq z#ccL}PE;_xdT2$*KkFznR;mBC(%-20{EG@_B%?}Ngx|@u!b97s&-L59?F$=nL|N$jvQL_G@d(M0tnK*wmKhdl;1Zx~UqD46-(-l^r^ zJDu~$WgQ2^gU!f?`?U!-@HG@*44ebeX2BdE5j|8TTvHR6BZv^5PPe5QsLHh-$Yx^&I`uEizVa3<7_+2?;u`T?v0-4?@u$dXE~YkFsU68{`pk)@~gH+Byg3{yK~JI z5+qN}KA@a#O;&RnytjXVCAud`7*4u(x)4ANM~WZn!Db!}$KmnO!_jezfJPF4)lheK zbvTL)t+$>~Hnj#o-W?BH>?i^H!L^K41?+BaI8;h`tMI9np!L@bWQ_`ZQv+>sE+XN# z&8Vvv+^*^FscPOW|KhX$(lUjn!+94VlWKV?RjDuS0Rgqg-wZGP!>}k>0Xq?xSZV`n z&$_%f;H*yx@b+H77vF4)9Pvg3$BSPm!qFw`r#|X{O6v2z4{krB;1U5El^^~ej-1Yv zPG5zl$SHGy2W&$v@E*cLtWfq$xtV zj>k~nz0SKg{oR3xcaY@q7nNz*aj=nkcAI~304B+4I=&B3MOh+_sK~`;fgB)=j^$ZI zM2#J&y*WO+EW)mvdBP_i^Ii(E6p}TjvNG7UUDdHQf$dygi0w92P4clVruw@};nY>~ z&j64l=eGIA&J7r|Y#Bvp=>1OfCip~8Ec$n%lf(9b@he!6?roS)^q+6!a#~N67kyz1 zt(h_n#_Z9z%x3&%65!W<4+oK6A&vn05|04Y09x64G6SLO%laaTl;ItO({(h+m~NDW zYxx>Cy2k4z&$B4co%%_nldHzf{OBIB-Sxr7_IZbcJ4z;jvuq}hJ5e~(S!6JuEs4pH%N*uYxQ zU8+X~M9_KR=bveA6N!lWe>wG0ZY)eP0nG5hb2U*R<*eyoknNg~ou8fvcIf=0C>>oD z|9E>^d)jSNbd$bV;a%hFn$J3(?8N_U3i(Ea&|ObsJDRoit}_*t4g#Wcvw2W#@9S_X zR)CC>m);O`EoYd=;eK~$_mgjF^gNli?8V@0p)q*({I$H&ySop7+8cDu{K9^BwrP1p zFAx`&RT_M%S^ZcEMLe{HD^ zMH1-mQcv9!tc}4p^fDKg{{DEg=LK?*3*gG8&X0(qKp)~k4dVzD0dDp0& zmRd3y5nF3yr=SU(DM>cLeAwG~Ig)0S!b}2kP<{s&zKWsG>Fyi;?k`41iCxmki`-a{ zsuOa6o8YcJa^)~%<{|Kb?x`(cAy2GB;cR=#JYLNA$&`TEmqMj2GID)nlkH=G<)eXy zVJJ3QFO^o&1qtm^8LlxwzNcC%1^ft14s-r3q)(R?{mv~LJsesz{=Jz+(V5b-<8b%f zkwrSOA>nx%F_VFQn$E6IR|S|0qj1eo;d~Ccp__Ti3aCPFhm4nX~tM zmGscw7swbrjZP~%UXT23*F#fzp+PSwrS|;^*%d+yY95iCnUMaL?*3yT8p49I89=D7 z!abu0>t#5Fb3`sx!`qMQFC9JWg-@D0ZiqAJI*Rq+k|Q3~1T*_1iT%C_Pg^nS61E_rzDX&H82!6BbGU1kE z4SisC@6nI(2~M}!U`mvRjgMj4dd6<42ikACAZxwFMnL3jIRg&Gv1~Bj?jvo7d_2{g z-Q%6_dam13n%RdEc+BvKvPgMiD6+p$N9W$BzqfY-14zbA2bOmYZ-@GOl2*mc;77ro ztb^E2p=J5p<{z2wd9skr%SbCg*69A3Jt-BCAGZ#jSSi!x(?XZ^z0NwGX<=YOY7(xEx#K zsSX;)0AdRY0(CWl*GJ>J%3z$Ejib=^dr^`SqD=V}^AHfHbQX_)0M+5Ke0i})OAsMS z);inL@P;T;kQUxQa2IIx6*{p!6DV5c2fS2xj^D*aFdq2d3S z<3D)^T;Y(a!wQZrClp{(Ai|-w`CDVxQU`{uLVWZ;Lt~Op!Nl5IG*w+o0!Xj1C&P;o z4&SW*9rVVT=U}B)9a#Jhx^%cfwgtAW6S`h#)52l){~p-gBjJ)Pas}eY?+Au14O$th zMKE04r=b1Eusy8>dSx@Yz_L1PmZEUnED2dc{ey6{0jg#AU&MU$@=+ICN#R=Y8Q39v z9vj**l7TJ1^opC8;ZZCc6IiGIg}n9XP%91Um5sdCQkQ|x#v4lKJUx83mSz~$e!b(6 zZ{8PoBbEaoaLRAZTjOsGA4&81y@SdA!4PLOP-Y`@#C)PK1!f<$02RjUTcy6)oZBR! z$AaH`xK@pqsAus4sXP{rhGXT4h$L>#3Y5#Ff>Hh>v=obulfi8sCJ1B$x#at5Un6>f z>#cHL^2%uhOt=@QuQcw{&xoCFuTS-8JXDjA%5(ofHjrGfWUcKCY&4j89;a@)*b%f0 zpb4QMJSsqn+x5+?kJ5vn{rV)+bGK=uhse-2aX*U~jN=%5A(;QO5D1w9$0DlRE|f(N z`swox-`n#YxXe2BMNXOo7{`zcs8e`vzUIrZY~BZ;D;m&1DCvuD<+>{G?7lxf+MP3` zaO^;wZmIBr#~H{7&-c2qk>y$6Pg{`hr?+3p**5Pq=yzX%PpvO3fY`|z@bk_49FQXt z3|}9;_cLvi`0hC*N#enm+64OF&L;}5gEiq~=Q{?8Lw>ORbltPqt1JNCmrZUj(&7G9 zaOd5)wJ7$y;6|29559)zYx~JJhI%xyCIQ#F#=qWn~ELCkZ5Ag6aMeEyvOg&dZQ}!(F<{l9oBc#z3_ijDOeOeBn`iFIY0?I};V@OeGRLR5Fc<%@Oyw z_rQu|z0DT!cs$Aj>5o>R_R|qdS24X-F#i~%c?#buZ*#m3x-d9Bzl?MxYjdpI`TdgLkeL8^mtlH_b)ZH>a?wUFV!`Ybzh>8&$dhoKk#jV-m-&e(P z+xm(N*Xrtq1wrT_RScDXSNLkL+a_cQ6L{dii4tj zn(I%b?-e_pr7tuNZaiC#4vx1m;xuY}6hp?RteVNMMrzt6v+^;v7ld%wOi@3K$&;VM zR+%F*z%~_-{R@fQ9kx=H?LMQ#Ztza=YK8h1N90$FOB;=^JVYH1eJk$&?SBXvY3|bW z0?hyBv7IF9JFD?xd4CjaD)t=jEtYy=88Vv*&zzqh>z)mzG-e39l_!K{ZC%r=W%3iQ z^u-f9Z^Rk+vEH(RS&9-s4pYL_y#(;Av%{55oum1X0w~59<$fOyDO} zCYkj;9A>2wBdqJA@+r0JwxgRQ_r@gWS5;h5&!jn$fkfViJ9{4qJIg3dzcwSjuClgQ zrjfR2Sw6c>gCuw=e8KAKHl;;*(1Rw^&KHX-F*Z%VLYRKTQPmg=o(oL$CQ@UgpmJ>lh9xja> z%RV7AOIJOBBiL&7jDjnqLc7qdh}!LO0wid!6JUkqmd@stSBY2qt{s7hgYB~|tmeme zseTp}FlD=H$s0jC5nnq3AMb_3Q|tryLW_E~uosB8W(Sb7sdyi4zV1>i(7F{@e?$~P z&~v(Qdn2$y%Be_GhBW0cHKDp%mLMB%xHgbgA!Fbb)uX^d6X+3Qs^YM1fM$q70_}nx zti($^4olCdtMPKX0ICG1y3iUyy`123m-|x2NKe`#S7VQQmUDYl%$si5cBJt`ZpvU| z0ayZ>xrWdy+V$Z~CNO2Tc|Lgj+MVp-9Tlao>1J}@1Vb=w+i`XjaC@11zzfI_Tmieh zfybclcldW#xzk}_oL9&fg-r4_uTDuCl|NXYU_dZe^qr?cSTtphJaz5ArmG`!dj%VI zseBluxlZoT^7q)WX=TXyOrwLuXNTyt0t^RD`3A>iARzPx>s{^hp5(;o; zDgr_fndaM~>{{10qYR?ZWpq%Wj5aiGN=~ezMig{8t4I9ds50l{2 z2NA%Pyo#&-E=aX>ScP^;930>RwMhw8+@jzf5?nLc2VOivfyzr3?)qB5hVreNi&Ms( zgb@f;f&ai4+{N8bMPP(TJ3f{21p(2=Br+0s3sqkrlK3~?UjGVsMoa{PKd->HL*4_h z0^P?L(v^~7d|<#=U?F{dnzVxzid-UHSC26t9oYd^nTOLb-(=}mt}UR-d`ELVadZMQ zLXNlcJt|SCZPQvB93kocH~=|~LE8Wg*14d~!#Ndd1KN{5Mbp_vr)7{Lg4gPEN(3i~ zXCTQ$ZZt`?K9bGmf3|cNXo}>um%Gi;;(iwqZj;7(xVc=g z7oLa10=;@BevI>er^dDO-ajKj7u3Wqf12EucWssj?exL1;LYX2EqVlHOr>qZw8e5^ zvO1oa_tEEho;cKCQPU&j-1;{SGs;I>6DkxQvrfyv3bsj4+00q0grXr%XpfWukO5Gcsz+3Jwm7@M*zOWr!K9|k#PqoK^_3rjgd>P-^t5x31K`% zJjC2z3N z!4sb8G1=zlbA0caQQgx>YW28TPXKh@dAslT68ktSD~eqL=5T!p*iR0;?T^s@a(rim zxC6b}ONuY~Ak8z9F?4(H(9vXZiISN#9#bCX-W)M$#Nj;sJT|(@8Q=hvR(ROm?8ld) zd9d0xq+ETE#uWTXvifoFI)E6R-j8d5Aog;lT>GuC-EJjr&*KA3a{D#$j)AOb-+KJm zEdbR2*vaNraIlQ**cME@)79r;AhO4R!}hnkjD}`1g2jG?l#dz>7XYxJRePmlh%(RN zV18@#lax3Nb}{76QKGHm4Hp1z&&Cv}bx-Re$vq8PJZrtNU_qMqb}o*8mh3KguEbtE z{)oFz8)KA|>6A;%fWrjW0A}9RE|w=Wmax2hLIu&ZWjB3KQ-YtuZE<~ZjJ!>{N)W;G zD#Xcx-?Et%Nj}4&^*Ako9&d0LksqIH)u_fahy;Yc0y~*zfnd&1)qdk$J1TyO%3g#i zEObrlJ_Wn$M!w%^aR+P27S`-^wci&Fl>Bm^^#JE1nIfhVnUvx1^BB-!VnmiJU9C*v zZw@^m#}mj~7${g_W*kC#qOMv#)-)7I=sl7*gcE2lo`#zDD4o!jr{crU;MTXQxp= zib^3vgb2`x*7cI_eN@3?#`srQWUnWdDFk62lFwX`=S-Y;}276ut-)A@Ga8~mY_sEG~ar1V5n=`!k;Al*Sw z+6YMMZ6yxHmvSBbwaq{RN<0TbbP1}sC{sdc11Z@7! z&~rpX)XKg7b?0fQ>#up^(BvV5x5z7Ec#24fC*f2 zB(?PX;@_FvfXQbHt49&`y&Cv|dFv$(a{}ay^%k(W_rFYXO?6S(hjDXR*w;V3EZBu1 z-yOtr{lXz5rXV=(`PN~fZA-jInLqq!)xu<&YM%U&ny%L8*V zO{c0Tq)&R({%%gJ-0ZjFG+t5U)&Wk(wLZR|^DmGK?v|j**48FuH}ZV$iA&=uPJ_tU zMaCNQ!k7^6t|(9MTXM~S?82}1>7LzI8zb4F6z4dU!mcv9l`Cw=|(5>XIxP(EhzrkZr^adnLdL+bWQcy_l27MG)~&O>s-yr({Y$DGtzJw1&O3X5(@~#>P-xO6LOBV*(Y)HE}{^!RJEvs(QJUh_nojX}}b+{?d+!(V%#Nik)9Eig{kB+s{4t7)NPne!;LAKgp0F3r7a^ zc)*kJ^*egLsZ$x5Kv~8!U+2q4f(HUuqepuHSDbjK(CU3mfZ7C+YsDnIG?z5pZoj?x zfz+cVqVo47sd20i{T<;P_ALny^tH+eZ0um0gc;K6d6yas!#dFV=6J`)iOSy;!WFob ze-tRFY6UhJXaaYqfB3U*z zL(`JmWT<~7@mBukY9Q#Z5AA8)moFo0{67}v{-=2Jzmu^cnR6XWy>c8vbCO1byniCV z43E4ZRjq^~zhl6qzJ$c-chIAk8@UO7Biighn=ZQD3*|R85#$fY>4TKDFo^AepuF2ylDlUUNLGl81aX24j3qMfl+<#Gv$v>6% zpKr3T=TDNF(**V4cW|bKVi2b>=7QuW0*YF83inKq^%I0g;S-hx3*|mIw#5k!>>M0C+h#~GfVAISR6uJ?rIdyi%5~W^OVqQIJ`VH) zEmavDzRG_mN=gnw_)?1?JV^itd`=~2?{w4!WUa?n zF(5HU861*>r*VCsc~^Nv3Tg%!2Xe}6Ilb!0;Aoc>dbG|%Hlie?13*>!N5eIgI-Y@5 z$uUWFs1At()X(wVa)upV4P4|vjC}H6!^IE@N=mX0_M#0+0m_S_lHA*jG6rzCYICwW zx4bkGpQ*U63;GPLkv@=mVF6t0QdQc=!N=yV2o&u=cNw@1>qU13cgVGye3$$^t?CDzF zMw5V9;nun*hJQb(4n9I6X7_(DKo zwB>%UuYHSh>+n#d9!SFy<68!MS#4GbQr97@b2Vv>#ZN|G$e}L!@4gBO} zP0wa7(kv}lO#p7Fe>YWE`rl4d0-_&fHfbqHcru{l+`64H36cZA5-Aic^kC4LWq}vO zp%rIB^gt72Vh1{D~5ujJ)_x~*5x8*j$q4fW*dj_04pj9L^ zyl8le!r-cK66a6>vJ2dTV@Xwp_J5q9O%R5F6KMHz(FE_{^bm^*LlrV14JtRVPFjPj z75^N7wV_*~1P-+n)SZv_)Dutfk9DJ08<5vykj#Ju{x9C%Je=yj{rfFLEtDx1G7lkS zjD*Zn<}oT^84{VNkccc~7MaVCS+UT75|&vCp+sdKOPMkc&-vALU3d4r@BQq39Q*g| zBD^X7~Dk`&XSSsk6E;z6z6*GkRtM@Q0S1!_1tDl4$OS~Wx zsZC%D>!JRk!l-*hW^fs!m!d^LC1nOfwEYR=zg?TLE?Wi8w8}L*xDOJ|#bM0^_vD|! zWfp!&v#nN_58pRA!@D(4dyoUN9i9?zcaOWR1pg;fvB1^)M2he&D3YkJusG%*59nPS?79ZX zy9EJ-3{o_MNIsz>aRW4Gyf8U)4zR(x61G3NB))p4h*#?T;b{&sK!z*r`4%Mta`-gY zs-+Y!YIS;eAR)#murS|89tg=Cc%6z9V6_qn-1S$<_xkps|9p5JM-UCNJt&;03@ z!T&>|)czfUmjQSRUyXrB_0P)rhe6qh3ci)ESPP9M;sd*P4xYAt z;yfD(e8W9GvQ9!0qr~+1<_D_{lQN}3xn{G=vaoM2{$=kj;}7KuDn@Qv+_*hBPW~v|m3Z6& zqFk#+9#*>WI8RUqypBSqoB3JJYd-CsL>EU{!n!-xW%Z%9B z!C-GRTUI@l_Fmso3t8Xc_InmidmT8yqlK^l!fDSy0>kvLcOP}1{21(-aL@+RyQOU| zSDXf1LeXPy*d0XS3T&1}X!oy|S0KW!HI7S993Dk+KHo)3>Ox6)6k`dxvijI-A4{(^ zlQkZJr%eOPZ;qt%5fuW#{oQci@9dRLovMbE>#^AD;w{_vZl6QmL?3CSqDAhiv1%q< zCj7DI7#T#bff@ao&R;HUuoo60Da8%`)4w>;tWTu%`tC)^+)3~~H+ErMA*eUr?(wQ+ zXzK&VkBv%A(i$!4{9xauNS}7xc#~wF;u9~lbO0+JBjZp}&z!z$Qre{{;xx!zx4s>6 z_wtLQO9z6)hnXc`j53>S`DE=mwcX1CO^N&NzIXVmJc*jb72N>ZEMoo_n|@7Cp4o)` zxO?f@guZOqiM`1hmb194J`8#-p=BwZ>&CB$$9iX@nT)_uyzbB>-0yYnYHo zdS!X0=TT)rT=z>qJ|a%^C$sW8iHc_1ul(RJfArg7E*yAkR4S9~*bR74SF6X!j|nYN zaSvnQFD?c?RVj8K*ikk$!oN(lod)} zVP3{_NWO^wlWc^2uf#_4w20Q{iSqoN@lIM68g%Nfq;Gt`@c5_Y^>mGMtV@9d2I>yg zrD_k^Ge;QJ&idv|MNMK!EzRCs=5#*Elv$x27tnwB$a4m%*940tXFXD>m}N}@7X9A} z1>+aXp0l@I*|qeOjTM+Q$@27_@Cf{2Fy4`{7r*`t)vf9yexsy}&|h&G71DJfP`&ON z3wqJy=9a0Logl^tcPnJWe=CzNa+7RGR@BF|$)a>spN1lGvvFFibNa-cFqQ4e&0XB6 zNW1>Hc7LbUwonBw<&4Y2jvpXNpt64ch$pW=@o6;i@%vB2`n|G+CnM-)llXrKFLJO> z93;KY+Ys1+d&KOu$_=d^h(}PSPAD}7VXcBE;bUCV`Wcqx8Y;>L)-;< z0)@7^BFou@8|^d+yZY&3RA(fB1z-f z&S}P3Z2Q_ARfYI41&Y;EY7a9e?mM&4;PcCCV3lgZjCWUf;8JAcxZ5HIqYFl`M`o zUb7(UrK4!kMAi|mI7ABqKiKs<^~n%^qy2y|$M4V5L&{JY$I!{qnG)sp`-QiFUP^4@xR zeUSLxcjIOpwVAT}8GhIB=b~oB^(tqlm&Mi}pASVir+fV>>(|~@`+f1(2qwRu>A1M? zcev1N&>*Bn_}d}vi|xMMHZ%M!SJ8YU=9+8iW6JPF6y}b)Um@GAu6w3NgA3R_h024 ziSqmzK4RhD=?j{|!dX>&wom!;q<8O_;dAt=3MU~b+$V0lrwo7ZlE{77xk(YD-(;k& z$7Q)W-V*R}`pN%;J(>p#OJw|i^FzONxaken@d!jGL;LNsM;9%qwKY7c7Oh@GdO&+} z5o&eEpi2L=<0m4OftB@EollS+Clza8ON|7X^6=VA$4{dAW#;F+cAy5W`3j0|yJEKt zN-mu0fnV==?sU1l8GSh%DmP|we6x#)d}_dP8tUz9<6jUtGt_~mEqEp;AdIF&EXN)1 zuBJP2Bl4VYk82@M2=>T_g`;(#=qmo=1^Pyk;^QnnkBUiXhFpw}Yk-4`uIJwhQgyf= zog~vfFOUA5-HQ%%%Q2>r(vsct$=?|be_GsgZlRv=;r9dQHZX5CKjx^A{e)X6)A#&T zy}3Ji_*Kht?w7*LFG@2lz|38_)~hSGFL=0YOV`@$ z&+5k1tF)cGQ=_==!=H>q;$+mt@vE(|9+yqjjb?lzqAkL7m-OGY-kgbBpp<_`n(&;$96ZwgEuc{-UxPfu z`jJN&M_6S)$G*7n^Q(SN#0kfu5NyV!C5AWlyzu0HER)M^yJiv(E`!MIqiIKzG9SV( zije~Y56`ekwe+rg>rc8|0v^P@wIgJ$b>o@>=Bh2!;*e_M^w!IXdz#y=ur1?};^x|? z6duG``r)D-bLJ=vM;BSa&Fv_$cMIc*-^LrH2p6f*$o?t2W}EK8i?t9Zq7eT-h6{TlYf0>xuop16la8@1e(6#Ea>fw zSSCp6bdMkR<9&HLWaaGW8`0!w6+ujp)^}Ju0+>Jpb4B&FD|@s?EzHa}cOKk6y}5!o zr6{}^+$W>`t{$rTUh&3=LF7ta-78FG)|(wDYMlHDR*J&FpE1%u7KDR?Emp%>TeA{E zv%a#2F^xG@v~xc^ec%@&gLGLKY50oAlHD=ixvn+OR(k5i=lAdwf5}vHG)qJJzIJbW zTJAOLEovGxq|;yCfLz*5$g?j^@KI54Qqglrs5N_<_O@^&<8BWl#&$dRPLbA`5eBtB zR%KL2nD#24R+fk2%*elmG3K58yz=(HW(;wL|7R;{4Cw=l|+&e_@OvfFWo6OdV5C zkF`1zCxJUfSjHz&EU}cVp$vF0KQ;=fGM80b9h^=fY!&lVY&aPW>iPB2Zua!X-bqTd zyAY||x$R32L33;|WoZ{86Gnw@d-*9fyRt=8e_lB45ko`Uu|t)Efc7w92+eGm#lp;2 zz_@|JpyWhZArFe@ZtAl9|-xz*}Jj|ZAH+5de`TJz{m1|+M zGR)+hf#D^`&FHf!TU<_lf{3ms7l#{994%pbgl>>zH!beR=rDS;Y`*D{!T&hOjNQ0y z>>-S81|Y?X(;n*PLFSL(Hs9Hr!o9Sw9K9LGGbLmyP}28qJ6NdS2qk6qKsSnoh&kkT zK#a8@rX1*f#WC~`M0PD79n8*Y47PO@WO+om>5F@$BD+vxDn~JODhgNcsUb1`d2zgT zYZ2Kh8gg#B8}jW{e-7ygBxf&L_5xA!<{Cish;CACD+pY(BP9#}lyzsMC-|6~(8Y?T zHELDUeI@krq4LMeOibd6iezYkHp+`tp-Mv^MunvkoQx?3uE`Cys1SQ^%P}@sju)8G zBkMhg#@)lDC<05Xd5>}p%_t_?M~={-lvNVf&59!F%zYHJZBrg(?^ZP7UT&zwil*Z6 z+rJ9lkSXafv$Wh9Ol97HFyV=d9n<9MP=M)?pVv*uF_==n_N$tD5l1V!7eTtIM%O5Z zAJ7+59K`(+-s_|~N>7p|2zOj~ULx!y#RJ(RDsP{C2@*r1!PL54_7uzY^$z~B zi7#?^8g#Wpn`nWBmUN^r9-r~yw(I*+z&!+9)OIV(9KS}&9c9yS?pL2-faKFUM{>qe z3>SHz@ahxpWL?RLo9M%Lcz0BpHs0I^2O`MXgM{MDEI6u3Q$x1&8P%Utg<3VqOX%xs zyo~0;2__ysl3<-^3A&!eI=!9^&mW)9T2JJ};e?M;12I9Tw(V3?<<(Xu^qG0G4t4>s zfQ}`L4LQ(&mr)^9pKQlYP~?FyMlssmDouGvV&>hsVUeoiIW+LRyfTHHc8>Ilxz42Q zEhX2je_rvxH5Iea%j0?bJprUpqVH+ej3U$0Pcd{Iq&#LQD`h@#VZt4^*1gN-JPM)i z^zgUd0^QOq<6Cmcx4R#Aa&EUw=79ZZQY=LWaaO9}4hfW1MqjebJ(zQ#i}dVzJGYL8 zZKXc%POfY#DPiTE)6mLA1svDt=gtKubcoaT;Q4rKFYte!eaf)Q&}_Z1q#QTd$vW- zI8nf3@3ffzG}Q_>gVPjIQ`%j9QJ1P>RJZv|`=HcS`d3!!=RR7v%H1_Ukv(?q_97EW z=6AX!_t9NLD*-cj=n;KBrMT0zF-kNgVU3J%ci#`FTCV)$49s85TfM`%YSc3-B91!z zlKvJ&SMT?wyOC8Y#B)L5Y!OF-qK;YqA)`Y?iXwsesP@#XMhdNO8;&iGHP;4ME(Vt4 z-OZ6JJ{wXRNINI_tH&%lYt6P^On|W8mqFV5@#+s&?Z)mAS<)#{be1%i<}GV{4dfK{j8LR^WKs#7!IZ&Zd+yug%I&9^Mk^iWYq0}bJVRG8 zV4^h4*IMiibDM(YJ}co@ETy+)QSvcsF4b5KOMB4^w5jAb<*z*)h&GIMPc$@@=S|@_ zDjk$FpZ3lSb7%J1NMr|_k=N_;x=`1%^Ic!$%;(;+OQc z2U}b#0?5aeJ#6pniZ(Ah8lJ3jhIw2JYuH9~Z5N4=3qaIUEQLS0H>Nj3=-aUrUgmev&LQ#vo2F#iGSe~-CIbKJ@9Uh#n|#?l@i zwFLNEBJ0_N*%Y-NS!7%{;ll)ZlLplh-ULWyr1Y{=8sjn>&@=0$;wGzXa-NG+ylQn+YCp#JnW~amiJZ~y z5|q73c&7IH>fMr6`&cutP<_(MgpU@b4V+SLjzl&!ym*2-hefh;DKWm$7AXp0m}?%f zWW93l-o`XuFl9UHDLAuoXL0m+7+lY zux>y?MlX{l9h){gw@l6~WfP@k$=7a{U$T>`)`zwKevuD@$)}Q0dx#_8N^az%>iuzo zo@HL9>SK7!n+fJ4Vb>kgS(xVEV|)cDg4|abC)?d@t@3b%N7jrABHkSj#nQ7#NMH58 zm{Zv-^XikrHB$O zaER%}Sp$P>BvPSU@iEmj^i0CX53nabX(T%A7#3~uJ&j?G>bXG5)kak4EG%xqQU`7%YO(|{(?Zb6aclVUE%I}uh``j{v0o5#8? zEpRjX&SfxZnAj%OeK#;!&80Pc-JtIe#}fiCb_;o0&DC6HY#eiZ9EXxRhJB*jyTLQ> zTyR{vB*`a+l2;w?o>pms$7;vj+l()fBifzd4T@`AIuSj@Et+Ui zsz>F1Nwv-Xa5|gfaVgTX+WCWIbGi{cC9d~ZLdq^ZJ!`+hQ1e7nN!Qps(KupJ1I^iQ zUpb5`*@aTiYXP-9uH^3jjn;g7qbu-&_E^JVR?l4f$M5nXWq7i!QvgXOLN)1cAfpb} zsHgRz%D&8g2?{zmYfa4m7dImN4K|~b?Q7Bl{|D(8VAH&u$F)}bLQ{nO^?*FJ0aYhd zrU2cn_>UGV{H9u&!818XvKDo8`G6b&{@|&8%cb*iBSz{~y69W`Nk|Q-CJh+N{13-{1?JO(P?+e77otJU<+#sWpw5R{ZUeX?DaDiezPYK zGZiAX4F}|0P13EYWTK^ibFT8cJPGreaUdkoSN79B5CqvmbHFdwSn43SMz{Kdhr=bD z25DB(jVP{O)K4<6^*bS6(n&``qw@RC(hwn=%B_0(wwT+WK6%geoj>jJ^)p44|Azao zZvkIl`Qn#l%0FDXeizgW5!%d73A~L6NvXs0dwFXFWaO-Tni@plZrC z_Nme?61+)jd5F$20-%n$g)8*oP?hyZ5}X77hMKq?KfN&8Xnb6a*((t-g7%$_D!uwH zqbN$oO>QP&=a-3y+nBP~P4IMf;dQ~Aga`L+8Hn+8!1EQD%pACgx?*HEuVDn~KH>?wgcZx-aw1^cD~#Cz zM~6P=VM+n|p6(A*3)eXXMufD?>wl(QFV*K1GEKS(BH9xDRN$tlcBYAIYCVvm)#p>g zWS?I!6+|4*3TMO`bJFjW7?&hzl+oa=?i`h%M5L|=G;~K) zo+ygxtbwJih928BOgUs($-7yGO__vOSy=wbm<$V`!$GpsK%^6LP_8` zw9FAeE0P<%A9MDc%~Tj3pWCqP{x7uD3ez`0r<(qdI%U5Nvip0AmNvFLuvm;7x`-II zyD!f_1a&&cP(gSpD<8F z&CHYJX2tKKrFLzI^r*mFJb|VrjEbvRf)WkOOe#+rv;xN~!F@wS&A)&WKnrA3{7wNt zksF0oFdcl#=bsN<%}uz938H!c7C19NIQzJ^urME|-UqcH7@_aHe-oGZBIMaILF$;Y z122hR&C(3{T8oEz5A>0a5tOjwf-bl!H(q6p6!t!Hw%0weYv z!^surWI?h>Vls5N+jvXSQr7lBCmFvTj(}J?cx;EyG=>yrq3}YT=)d-?4%0B(=s~bL zUEqy+H9_R4DiFu(eUAxFhJzStX&V-xy9Su!9*BE)gfy1)mk$%K*45J-l&IYV{>}Y` zLRCzV==hftz4_XWMQPlM8{n0`{H)pp3LgRFOG9^5>k#5_1GXQmq*% zi$>FTuW&DtdPxpY)xWcXPG`P;gE-QV-cKHG&YJZV1kycn0Fy{DTG+a%!-$+ z)7?&mJ2ZBW0vbnZPv)Ndany)5f=n*)W0r~113TzE9b*^nFzWd( z`D^vJYfZxS za_z(fP}wkF`6sr~V<*5B)qm@?>w5Sk%IK}*$kE5bBd0A$Tx6%uk|aVe3@o$uzy40&x+oto+wYJ5`ZQOt%(9ZLQDd}E4IJ8f(_)btQP9*3?z_h#zQ6Fto0EA*4 zdyuTfPWz4_R5bcf7Pw{7W0rOLDOCP?fA5fvL<_4r;RjK7bY(Z1bZ-D{>@2n!3?5DV zWv_CS+n{X1bD`9%2=Ddd^F@Tei9j4d>a`Ewlm7dams2AAGD)Ch^*=pK{rzk0%M>xY z7}MvDsck$0n`=LXz#o>sG*GC|iTE=S5S9WaJ8l7hHK6w;la7kVVs}NqeX1<@V^)Zw zI#WcHjxdNSGr2WU&6R=k$_jyQgISOUAqrH>+VNz#1|vr082~B{z?Js%5|6=clNBh6 zqLzb+B6$9dd9tX@A(jiw`AV8s#S#Q5_(HJP7D6;&ahefxaQq$c#Wpd!a17%MGI)%W zm;F(<7qAFr?+9dD0*wis&L{ToHg#oN%DL3Sk$TS=yThDQzI6 zy4ilY)~8*XLbY7^x|xDd9Xv;yuNKR~cO-C$YNc%E+Lzwrb=MzMKCJxU^5n`JLmkDcy{)>v z-fSM740?{F^*g&@%j~`D43^hL9%ze_b$}pnG~;!L03lSs%(Uj z0`R&gWgD@;lt(7OYQS+kM-;+iI|s03L@huO=*bo5NdU)(f)@aax!EPN9OZR^M>{RI z5so9xQc;Ttqh$_vnAtP#jNLmX7jQm&1;5$c6~EcM&0P>mjn;y)qUEJ5IjgLhoS|Za zp~dUXuOPmC1w=8uVZ^Xnn;E$KiDDVDUVc?!?)whV#Hxiv)FK=_W`E7}C=LGvD%lH% zem=P)a6bC{S7dy`Q3ZUbxD9b+!RFqR$q$qNjJJa7*$3t=1lr5C2j_;@^q`8D-nK({ zX{!|R3jjSr>Kzyy6qjeFDWzMkxVN9!{Cb|}fsOMrjmZVRf%^8y&s_j)E>$5PzclI)%`NHDEN7MOtv-6d^CZ0VV255OHMe8Xii?apYbh zrG318TN-TUzeuSclYa7@exTjW*9vr^c5u?=m_&Yum$z9hN>2OuP&Vg5y8VMO_1~zS z3M$?L914B>5p@`gU!j1-YqP|~Jj6V{S)NL&Uvj?5Tf-9``1?0ntO#Nqnh`wp)q$P` z5G-6I^UkN=xP{br5X*Jx@zJW5)QzLMY&zp_Ay90DclG5JAY{5O*Ld3B?_^Z!1fk4E zm*q7QB`xvVcue2yO+B9Bz`a`w?~!$tT+FH|JZh_|zM>b=EXbMQA9Z+ufXMx?EX!1J zoclAqd`*HRqA?n@sX=l6NzZ}+;7!b)6VQ_A=f3!Ga6&`wPKq#jiLw?;`8^vjZB50? zeg%02BGE`*BqZ9hE__=iW7{O_Mh#NJtWcZkK?2Y%)K) zK+$Tr=Z!m8KDP_g`{KKlnLJqe&xY)Xz5DAk!&4{k1Co8VpT(g`!Ti=QV|@-qi7%dp zzHnS=8Zf=xi?Ds1`tI7_iSK!lFVkd!)or|`c;*{Z+P1&R!G?e?C(?oCl zFhd}Lp5)*9ia23R--mRF1|SfNCyqR*ILu(4+)KQ?!5Aoe@&aQ^`0J!}yh*WpZjFw|7?x=r{hw5kNa9*^%d5`I;wuxudEAHrVF!_b~Q7cbw097AsZoeTLB{F;p=o>CvJ z81}xF?7!g;^QA*A>g0aDtma>HsZ>G?eAgwfQe6-Uoa+N*4jj*g>{|GL@L~U3CM^Vw z_K7`^asAudKfR0Fe_Mc`;=%#gUx;`Yx%Q!I`yxC1A5gJe-2Yq(FB{to^hQq&M3eMK z{PgcByrXDSQqJE6FeU@ZOgR)8(|w% zdQSn*C`)GRd!;DsA@+TR`V1aTxqs{bPcLP{Y6X#WZ#gR*Xf_Pbj~tVTBf6lWQMm$} zzr6?&Cu~5;{ub%aSIau>*k&X&Kn{G4Ai!~ldA4B>ZBx{<`2Mo0cf(X+t}VXc<#q#T zEzS3nvhQh(fPyCuj@~oh`vJ)1{ngh_!H$9ss*!*qTKPF_@qEyMKUMZtjZQMsALs8w z`2L~!hc4yM7R3=rHpW4cMF&D1ZeT?v0ryq|_IJ-g2sKL~#KGlmfM?^RxvDt^8EbrC zd;L9L&+qqn0hH{|)*YA^p=h+n^t<~A4Qi?@Q%EyOARa&gMo1TuSy*gNU+b1H5cgZT z<_o7C+fh7r39xG#;LNfG;K=(f#N*A}Vnh0z6~eeU1ck=tEnHCnd$;D#ueG)ig7^nhC9=DdZ<4>gQnJ)~D02~Vu-t@)$rBN<%9a?h z7{4A%z&gf$==qb^M@NH-ki)r>1*d>iU=zg+2q!uqx0m1b7J+QQ3jGWq8{kZT16z1; zqbDR)_cxw}+qjE>1<)2&s6r3{NRabj4V*)2Y=R?ktf>27VXa^5Iz}CiwDCFN_KTRR zgZb7<&pu(pf9dXDgbg4p74;n0DIii%xTs|Mi_9Xd7P1vG{U>iCT}rNYri{)*mT&Vl z^+5(VS)iG1?kXH+PIMbKE7T|7F?-wN*D<9uybE#0Iop|{pPaQYjO%?qs-19Lsf2Lm ztD&m(S6#WCiCeI}f6YCg@Cu4%m*kWf=uTal2s7k7V-D)jUzO}ip*{{8>T-x|YiG+Y z^Vaiyz{cMwPylRne1Syj{!S%9`B74=4oijN6kzcgIrv57C%l`7$JyJ|2V@NEwrhd( zvei|542{s;$bk_?kz);cXCQ180%0Thig(W}^dPvkD;7>JKA-(l4j%9BEsq@4^DC8( z7!mgg^;>};u-!;_*s;={F7ybf%WMfqxKRb@Gg0m!wO1PMa z8m&oLwPu{uqsXBGju?T*7ftv>wbg!9NObhzm0 zg2Ge;#tixitn8#a*R+4`kmQgJ(bser`vJ#`G2o*&#&}<2Bzwb<+~jL$F(Rd#-qXV2 z=9MsY1Y2Hy`%7?|PIpVnfGOGP5eHIbMs2*LSaRhRS^~~aq)|g(c*ygcx8D*SIa{|r zMs^nIgK#L*74%w_W{{G64|#_MgF(lFd^sC%5%#{IVs(@l!F8gaGeT7%4ic!k!(_XL zk65M7i8JRo$p&}r+x0E$FlvzG|2=I5phXFp4P}WIZcs6rX+7Nu(@#RwaWho#5aWT}b6iU!a~k^2 z@LW~VxPRax5H{SK(GJlbV(U0+(F-M#71~qiep-QNv@kzf8IZM7k zwMvX$`$b$k9%RvHK+0?M$Y zN)kea1|9Ws;!bdZBV#cQd5L)U=(w)7%X(MIF*HI3S`8ctI=s32mgE9+#j9oJwJW#6@0*d&7QKD#es1g>b@)GkJQYXndnEDlZ{fv8 z>OWGQeOG;JYxOITjqb+}AT{|+2n)R#-kdG>KlVUoHMxz2-4`0dtcb}3LXf_*j;oo- z@OZbFX&ZU45>IGx)wIHaNtM5Ij=YKCb1T738%y0NsCL|hrCLm3g=))^cZ*VYo~5JK zPPA=-!x?$WO-e(iO*`%hg;PC!bX74CC5ky59X>g1lr1faUSosu$D}V=ebhc7ypl99 zAb!l7+EuvJh&fR80tN8iYllvJbsa3dBCXHKS^G#ty=T=Pv4{Q5hK)}`z(?9=u z_E^aoOR*qN+@k_Fhd#xyMx8VeCifqDB-wg>1R>C0<>~(Z;0(sX4QQaFtued@xg74C zswE}9e0Q9g4`Q|c(4v9%1lR%gM+4IDHMm5vV{yYK%@+l4C>()nyjs$zby@{5@U~GO zDkuX|&@~>bF1rE2r(X3tC;O?8IAe^%1`9RhnyK}k>0;63rdFgcfGcPpMDR%S@gWHK z?wXGkv+nNd?o{*+g+fwt$WPiOJP2e8hT#9tJ*qHzH25H%bEeOn$*(_bFitbK~4y4|>Fm1#sN(}L2G5^rcCQlsDznzx>1cS%#gY2e{JVH5K zQm%VR_;bANN4?2oQ+7%+pBHM&Q*SHpOxIrMLK^uQ1L@t>&T`AYkLwY{!_x;mQIlw_|D3sFC=I!v!8Ek*oi?uRs zw#LOtxZ;sG@ly6baUzZjVYKQC$tGYLLuc({p-wp$=Rh10z{t1Sdb?h7B2G!bI`<5< z`XR*A;`guSfJsDsnvFzb1Z8GnPS{Jc**9hZSjTju>nYKBm%HGJ#h9Fczz+NgE>6wS zzJ|$6010~5{HjLvK#9ta+BNgqV4_1ANC!9x{CCLr6xu#2r48FSaqw^M3lf7eXd2BA z{hp5$zui5UzmDQ31te(=2L9)o8Fu5XAXO%H;+0|j-;m|~9t!vW?xBFhpTB!3{96gx z5{yaA+<*Cg*n);p3r`06Ul4(*ePiL|{v{tEu>tdITfP1y$5R{v0E>AFiCtrkX=s!f zWJ4*eRF&6pwChQ#R1ckRGJ|IP()=^ajll>{g^x55#-h<982cSvfz54fgnv)P>L~`AQJ^nlt z)j94)#v{53atP)-_E`asQT5@+S~4(9QR0}VK;D5b{sdQ7aThv38pyMCwh^oG8Fb5= zDpR8r*q-8ipeumak{Xkna=4=~FWky!B1?~;c(TiPbm$$da)Tvk-9FL)e8ClYtT278 zpV~($;b}9o;a11EZ|4H?F9&9YBBFe1LaYPOXRzZw&6HahAWLCD`eXX9A1F{Pv&tDa; z05J=Kp$`XMsYWQk8$D*QvqIh?Xi>srPFQmQxd1%bdt>Xe=42xu5xn54mK?qMmgceoLeAemRUuBft;j z9GCQ0CWt|dkvT0uJaICp%qKM2bbtd@CCSwvYWrFzq z3vznJaIu|%3SQ+WNYNx7MbHfqaC`c0#vKIl@}ux=q;@R`8Uk@lPfX!oa>XIdxQmLN z(Mix569z?!q+8CrED`dP_RauB0cr^HK}7tVCxVm_k22hIRF*J_K%G!QM;zy_P|>~CHlAzl;$5C>gr68PlGGZtzNyjsZ z8T6@q&mS_$8FU)HQ(fs)WfT`B;)eH;6r-h9KMj9VHlYwxBhZ0&=UQBoa*>+%DE#Al zSKG&2ajFEc79+}!YRGPKMsJ&5r7lFlm(%J;3lNb^a3Q9A*hIjTpK#m--cbm2`KBUe zB}|adK|JBWF0MH%?D5AJB4qkVcRjnb-z?CRE(Zhx5V}a=%$)5gvOs}1{-RJPEr~MH zN{BQf3QqHXK18_PF)&i}-hJoR{<|$VVi?(wz@n*8u&e` zZKV;257JM3MT>!xgbQ+4hz{An;o$~bkSP2HFQ-4ZHo*yG8d=pxJ={OklSUS6`GgVi zVd2;E`ncXG5RfBJy|5z1n3@-9h8|Qu^8nUqWEw6UVIU;ox{thHXmychi*Sd2Ll+8z z;Qu^&dp(7SX&Bj^kw4QODH6XMDsPLi1V`-W5bGobZ3 zHH=uUY;?-uKL!sp5`qy|c!-=e{`NRr7*m=2)l4EY$bgMnJrU{RvIKKKKIRz0zfwk8 z!YigkMG#S|0xL=6)~09l0u;b?=ryiFH{yCk;dkofCMPs_hfx&; zeQbojOB4XRPD4II#gT@88;BQEKu$3*rlBr={2gS%SQ)vvm681%etgYWR03$Ce@)`^ z!L6@g`~DXLf=ll{JV|5jS4sp$ux7{C@Yw~Ohs_Lj2n4cpTN?iMxriS~EcbL^p{|0X zg(m}48`4)QL<8aF_`)EaIM_JErFZc6h8+oKL;kFoi$wpIjRLynKAxkeBB?_BiC=20 z-<{9mhIDr$9POrL z+bAsY<%55}E|@C_u!`>{v-JN3>PXD)gvyZl_ zG0E!`A>qn|FIf5(ewHi9>i0{gxomhujer)R`2_PNsIUhKz{OpZ_W^^po*#0ZcAYfp zS!VEztvD=t>Fl*WS&G6#g+DP=1JJlNqeBF%qSdd0p*nVrTM7AhT4<=9vq^f?$r{lS z&Cr0Fs%*|sW5AG8U`l-EveUcJn8&MbEsdBRQNfa0UgKBlPmSE@sD+jkSh5zvIAh05 z*^Jno%8WANV-1?GsK1&xNRTR6x+*dnGUSObB|#_>kRPq|`%*26w~`G}Id$JfcgL1l zNKoF$j6gr0R(9B{XiUYAZrH)%aB{}Om!GSb;8WTT`r3RT^xbZF6z3h*Wm_aGgl#Gr z^1Lf6G_j~Q#GB~xc^+sI@#m{nOn@dqUoPxdKhgpMVyT(aS(;IAXfu7yqbf=TWpnKx z(jB`Y?EiWjE_sZNLTTv*CwwYlU8nXS zoxdL3MFv{px(WSXThbO=SvdK~;!kzrKBg8I}8RB=n-;uG%;mF?sVb8)yN>s&l{ zU1;(y!8^m<@7r|EN`%E{_f(b(*>{6|aktjzr(-Rq#6PkQS{Dtfv1k6|%S=WMg-u~L zHU!gvC86Za%{F0)qkl@qU#TXszj6=kt1?(75=$`vdPA|;X;uxme zp@p>>GNoCLtmi(H7eQyA%=bfaOhq=hZqD_}1AZ>#Khp9dUdH*v$11`r_>65zr(qGC zO7%h!q?X3{J#Lbip9@E54c9PKYobY=SIxKYQ=t_YBmEmK=GN}B@S^fBv}K%?wba|( zFcaQY9BW3^yVRk+kL``q1$DH$$f+!`P?fngY^UCA;C%Q^h-th$)LO!B{2YD3S)tG{ zlIt-WV?s$`cRwS!?zmP?&loUjCXN-62fjK`Hny(m)bcM6?_ z8v1^gBe5&XS72OtEv8nWsfF-yVeD%|@iMz32OCn$c8*&#Ok)+sK|P~HT_Q;vX_d-QCnkHelHllNI?kJ;-c50Ha#99feVR=~i|4Lm9w&44dfQasP|ID&B@{>|f zv|vo`(4b>#^WK$Sn%(cqqt504OP9Cu-x|z6(U;x$F`!O-U{=d3AVN`z|AK}kQAr3r_2wsQ#Zg8P-^0TNAX!D+2Y16EQ(Vs9v<;zt>pDsCexSu3) z4u7JgTKEj|6ywa1XYZwp(GS73i4z+nNP-H;fz-|DS-z@UfXq`7d3KcAL_ST!T0WiE?=;^M}J_&#a2H z>|ZkUwYQC}@XG|Ig4uIgN?J;~{%S{z{)0l{L)j9G7m~R%t5rq-ClLp|+w|@HpbB*l zF{nQwP>vgdn$0BuixgR3@^~xWjAa!-n(ct1*m$}c6i4pRz)n>@Sf49izI;cu`VBRL z&(jA-!s=;t*`T;Dalv>F3qWkV)8UW)h;+Kyb>9P5`6OrwlaWdaqTSv>WGT`?Hej!} z3b=^$;1IQHLMf;rJ*v_7_aGCEBGSlq1}LdJmf*tn4CMHQ!g0pNUbfGg<#+H05o87 z_>IYH|1}{{I_7BPy_R|M-iV`78fmrz#ipl)p62d(*Qb5B0?f#M;752@RhCctiZx)J zozwrlpxH5W=owQbO65^JX(V(Yvg?yxLNIqXK|kQ!vIFW8Uk6XF2mugw$6I=G+_Tz> zcBd0WlFB*t$0@wGC0}@n-z7b!TfUgSU@t|^Almd(;Dh;1r=P(8SON)_7;6JCLZ*t? zU5C410M5v1jv+;ic(nL4!^xk)WP%HC^73VhWzF9D z8BD99C2K~P{U`u)8A{On&x2^BX{`(cpcSLuW~1%`U1deb-nhcZ5k&%zU-aay!=I}? z2z3LcA3(AD2GS=c-Lc~C3+RCAgNVSw*nc5?Od~98Q>K}rfz~F)bVqlOG|)9xghmk( zNTUz}c&Qzqvcu zf4{>KIiVKS<|alN#qiYCg|5Ke-oQR6?#byQ)ZoD33`?S>2}C=0=0%AXcI4$G6u8$C z50*a+ZaDoJJR;LOuv-PtLz@WUoO&iQDmn%Ipan|aJon~R#=H()gYi5ZFYamu7CTQ9 zrY(B<>(lI&u*}jcFE^Rnf$C!1N4EwVhi|Qz*L|H-4|G?BEO zT*n`@Jb^Z=Yhv{1Lp!M+QTbVoXFm3OMTK{GHxcv_f;i>ZxcFkaH8ks}1;xPCiAqWg z)l+)_7=^KR!Ab6C!C5f-?7coq;;zQyXr^acIGO+O68l)I^->h4)I6Y|(=9iBfS9o! zpf7|-l3JG21cMLNS1X@rDrj#6DaZCqFekdrr1I)p|G*Zx7^H-;20bn|M}3wK2eI*Z zICBbCs9;I9)H-_}>YENMBFKV?tqy3#8q;A*7RjrkY2>SAAOmIN8!dE_0Y$(m*(-lH zR=4|0h=QMS!8U~bmZxh$hI_=Wt>fz~%at&FlJQ_XK>*}e9;rzRb993zBWfF|x^kz; z9xAY<^oIVCi#RLdx?o1o=u%0LVzO6F+JvpXK5S35DW}~%#!i?`y3?UAfO!pD2 zT``b*$Qy#aOUH3=Rb*!1j-e3nFv>DnbP#Cix5khmXb6HplDa^UXB%4QPp1#f51u4d z_`&2(zcE)-;O9_X)^)hcQR_337e*T%`+yPc z@kbm+X@_ZS4ohdgkU2ye2?~iOx~uAkTPj*qP6z-K%@eZq{;IXa%SQt#T@-pk{4suH z_z94SlzuQ3BgGs{`t+O|>0)6)0foF%gM|`p+03-M> zG|RuG<<=rR*<7gNok1z!=}jaY_`qd2xDJ}Zm|DYx`4j33GIMBL<(JJdbfc;Dp$z5ZoM@S9@n3Pi4RU`%IZ- z%1nmLGL<1iWX_l=DwU;BC>au!dCZh~$W(?1X&IxGF^NP)EhV!^l$ku_T;GMB_C9;> z^V{e3I{%&h?7yC8S?gZ+eXs9*eXr~Dd4Cp%&%%D;9#$y3#5GDhD7C6JlhD6_jNuHO zrt2ZH$my=Hz^_>{S72NVZ#L<^qpt0skPb?tzP~_?n>HI-z+WQuiF&pFm}7p1!L=(J;g(XCE{g+Bv1Je_K=58 zW{%yGdMpSfoS3lKl)U+DqGQlq7O5CLGmQ(6jQK}|s|k!RL%5Z_Q#i2;-GS;)V8UCI zxHcbr=JOl(bSN07ep(MV6z&?qa=h;gePb`qoQL&$8g|6!;=v5jgU$^BB+vHf?@6H} zWykpMK8Rx=%(z-|W+M-XIsVeN>dNm&t;5K#Ge>f#UWQx^;#0x^c~Iw&v{scoo;l7< zJG#5L+-iJ=Jvj#-On`ZDKm4Ip4in1-KOt;sjbE14@LrL$C=ph}A?m4$UVzBd?I$<9 zaPCKQ$~Q_;Pot-9Y7An>EbpA%KRd?uRdv*uf^scR?rRk5egXUGvkEOM4F&p#Fa&?? zT6{*m(P=mGz=R?Kpc_AbhG6m^?EmDQCOuNn|9qHKA`e?V94@-1M4F$m*6d{x>=#uX zW@^R_`w;U&;c;q;h;+-TOJ5k6?GX3jaIC*uU%Q}?zWkH!chi)zw3Z!yDvPqOY9(j5 zDBs0ecMl`EiBF)SWHjVv{J`Bjr8vKyTJy^$_&FE|N5@BE)M;L(4hRKnN{Z_*j=JIw zyugxwOg(qYGUd%}E0@tH7P^?(dZJw`KU=uyF5|MAD@bvLX5-is%?a!-tneS{tddSn zDO%|tdgoqjXw|$=q&@>AB8$OHOt>4QKtL&A(7X{of;{;dD1^rF|>BZ6NG(E7i@m1(|gQ=zl{0o97Yz)hg+N+%Q(LhhrNv}RtA*jKyyJ1hCdL2+5QTtyb{XA?~F z_IWsQyxkJhsOKlUnPhk1QCtnS4it{(O1iLp5?0`NP*nS?`RF|&r z0@uC7ms-gH-1C9_QcQ5q5gopX`u>F7q5<{EC6a_%R@M*MaJYna&0gn(A}W_$Jq#Vs zn3}ePkWgd83u&GC;3z)X+l>-O!$Lrff88=EOu}{K#oA+;`#YBlRYU!#Gm+M8m!8(p z2dXI2sI~CjOgi%j^)(-0%qX_35gPq-*{P`!+erBJuG<0I(1OMUG7tLa=XZL99q7dF zyNYFqRW1^QtVApF zL7;-I+d&i*yo11W>H?;*>5D+MqKIsTbSKtZKB>&ch=PUCx#Er1ul*}81;u8ld`fRo zeB`;O96dcFq7)8w4LkBequ>kDrWE(DQB@CBL7cyrzhz>4HUUE|Z&Alcv+d8nRS zW)fghk+8&u zkm-xA{Z0as37i~v>zS2;2twX;{KCB0*$ zaZtj(_9USxAL4$@(*|QDZJ&o6UCG-sH8Uf+3K*<;U?3@ubf{=S7Cmd^A&euPKC>DD zpFu;}7X~JlF;95NDNU7PhU6RYssA8ETEteQMBJo1!5qdVe(Mz2EH$Hg}a zR#IHErKUSWvwe6XR{2lrGm?ZFOVrcb>hU_H)LF}t2r`k~>6a9BAd)sF)Xgu_P#e7e z`TjNI+d_(@#)L-_~$cUxm5?-1w$L?Z0UF{40mP-bt-~Mn#?GNgI_YtR86!Or{ z?LV}hK>C~Q;0x--g5Nja0c32)G#)y>fbV??qd}0x?6$QP-*Q6a{VBE`|3gFq z2)m(%h&H?r&5l2klR;|&Q*_SX-LQYxb4#Zvbrjz0>IkDA)Q25NIsH$F1V2y=;IJ5l zyu0e5|2CHzrpb^?Z8qKcpXEE0yjYv(6sZT@>|RR(Of>;IhA+WGB-)WC*ZK8DWg7Bs zAeJg9{e@7Jo@1}Q&ylUU`RpDfo?nUb7ktA2d)__gpvaZcss&1EE7X1>pzyp92 zV|gsW7K$@HhhwP{^g^Ih2%iYNt6FgbQ}u$;L-4k`33USPd>uCO1f`rIi9c3xfUu{e_mJO)QTqrOfL z6mj1NI)K$Bphd=Ea*);g_|t2ZYl{zk6Oncvw!l2+mm&8dii@?^q3FTroj{tnYm1Y% z8j4pY&5mQO{ik42TiwDbAm2<2yR#TL=-#>z6G5Zp;T&v;O$QVJ&zc7-GQug?@UDQ& zcnti%$7Y+j%w=TSyc5kC2~J{4 z!G~2dksVR-Kodd)A=U3^grPGqZBNjm?wBhWge+dFAB{=B+(<7tn=|^mA`z4N$CsV69AXL zFa}$A384AzzmE4g1Tc?Rdot56J?-vrF2p_eSKO+nLfb&^OLK|fS}l&|B)5r zPZ@a>Wm_RN-2-yMt1~|@Uhu>E3JJbqg)Z)D={bs-A3iP0NWZQj?9rpT?gbG znuCrEx9&6y8pyfyV7Ay8?Q|Z432~OF8KwW57ckU|rDK9tTwjoY3zeFv;mrl{q4trdo6oJXEyMa~T zB)ufH^>4AbsADh%GnOJ-s$X=Y3HR8?%!)u!Y|I%0TJ=*1ZhIfX-9sJ;1wDtJ3qN!{ z^|WfMQ=mVk1fN{iu#-lsEs^65b&eemft0U} z<>ec>9u?4C2fUC+>y7cY=h~dnA7q97+?0yu*(1%;hE8KqU&+p%#L(-NW9#3(J$b1{ zGpD|g9%p|}ks`tgunGL=4*dh0a5Mv}8id<33|(kI8H4E&f*?a9-QS9dNoKo2OtS;m z`d`bPxSS@{Y^>y7&qxSG3FaJw#>KbKL-6Z6`^Q5SyZ2~3R`meZ!tXO-zkB#}Z7Exm zypsco3xVQJOAq^7=W$bKjl)h6y)_7rBGGcUPPv))0Q!sX(lD&zU!be0Gs8mq`wroa zZWGgFgQukl);c|p&zesRTQmxOlw7$NlYKSiQ(%wq3Bfuyugs1t%E*?r`ILTu#aR&i zVsAR<|2LLG(!OGB_Rb&2{YMIEqx?5O__7|tX+=Q^&B#i|RgrLDHK;bz?xLl`Jsbik z?>lzWpB zQSIzt9X%qn2eO~Av?gM&ZWn*@>USYx<|zv_QAs6W{YryH`?Hr#@{w9KP8ttlY&r6G z|H_NLj%})6>cd^03>Piw@)0qI${UyP?WJH=0`#v3O_KL~id6Tb1|RJJNVSulK#abc zzM*r6ajVXRiBC1;xfPZID<&I7P~X~dPi4l=1jh;(BBhb%nD2@AsyWmP8mxDZg;qD3 zy)m~qmB-hl_hyERUV&a{_vSUffUZWu8YoE(ujx5iWsGJG%Q}U?`o#QTajEr5z3L2YMw;MAeU_o})m&7jwY9 z!Z=RvE(CDW>y#A%v-SE-&yW2hi&xRzoNS`=$09;nt-0;=fOXPi(|RezVA^5qCijFnILCy$?)$}i-v=3 zXkRDuUsT(8N7P%GbX@vlY54Ez?VG%Q%XWhntpN3uKu$5}?~5A$JZqEp@5qA5gY zl8p&6BQ7_l^@l0RO9To*LUi|uuEWU4j%;pY;J!!#>3=K0#gJb|RLBrf8?f7BYkYZ| zd}{H-6!=9<++I%Pn2;lO5yaW^fMNC>ANbHqgT)QM{pi`TgF^&m>=cfBxq^A4Er^`q z9F(;Q@SHRkAxw$@TlVH1fxjfS>vyDGt`kCukqBfl%IHASU9ZW=PDY>y6MS1{FW~5( zgFEe&=W9xsCeNT7kY(FMDxydbsU>DreU$6Q%;8=D;uMXJr<{$BMYN1d)k8ZEuFW(R zDRo{MLBDVqGhR^dg8cU*;L^#3=bbBfa=*-#{CGOkNO{d{6q=|H4_tVw0VDzTgD~o| zg~h6s)eeFdudUi4gN-kY-;%(fatwC#2YS!N8@PrH8Q|q=l*nNK%M$D*%#RUTlHg08 zNVzwMA3-3buFp;?7H))wAAgqwIL}PJm>`gtp)Gm)5pus-8>X( z&=yN4N+%unID4;Hy&7)^4cev zhd#_2v5Zes@+w;@Y7Un>+d^eTzQY-&y8SS6RhEg$&O#PB);CJvXv-}7Hq)IDu?%>L zYr0{c4wKJK`M4CQ8o~YtS=0OhwJ09&MevU>Q~;|$6C*R9-f61+%cRdY$1~;Vv+|xrpp_xza>s&hOWOr2&)Ax-}rwOkheak{PmVk2qq+=K)X+ zM&4l}oq=*@_Tnpn1|*TnQi&LjdnYBI`d^^cx2VU6?tO}g5VrPh(h!Sb zmi@j8V_Y$yV^sFR!=nsx)6JNJI(6#F(NnC4@q}dxkD=w&CtCm;a|8KylwJPRfk@5z zoq^l~!aU!NU(&GYda5uI#9&x3r!)`+jtfT@e_<+pBXD*h%OGsP_+gLou0j0Sip|qz z#v)DZ5sND{0okywU8{Tn=^PTkYka~a+7mpqpZ+p<_SX?V8tJp5PaDe|f~}+3n7z_> zx_YL52V2d4IzMD>wF4^9ML` zuDsD?zA62^f?Q2yZ_Ii0;3-{xhcR#-Io(1pTPILk^!_R0WZbH+_a|Bup%CkN<>7Eb zU4rC!Tqj^yc>i)~sg!=Tkf1u#t*BWG4dZ=I*5d8v0%hvtxUOf)Nq{G1Z+qiUUt+r&lH+h?tKkAcYfhr> z+L5+dhY$OW<>RFj_hehjL~!uqFJ3R|C@K^qkqu;sN^8X-XnZ~VDEkgs;WHg)2zww-wV zj()m;eNRiN7^|oxu|cH>hf+W-A!Uo1pIm@M6eB~0uX~?r$`3bs3`vqbPa2f0ZZ=GJ zs0)vL(1e+~dFp@^XW%J0Obq#1BbnPgdv475TU!|5Y|#eX=U%=+ngDKyLD5}g=m z3$8*Le_jPRD38&W?wK!sO+nN=kL?Z)f5?tVp4n^eWIN+kFvxVePQwo$_52#Pd|LEEZNS!AbyT$p)fADCSJmUvC(t!mgHdb zCyP?%ou}hM)}^e&C@@X71&0JO@#5l=I5j*}Y$8DxI+SSh2@lCQ+8eL52+6E>g(DCW zP+&4j8_cDSBMUN#1ZC5>{Pp>w8?zWQeTBRHlCL1#7%ym%nr{S z;Nbb&R`)E&LW8;sdRWG@tkjzpH2*&>XoK1jC}2^CI|Dt4Co_K6jZkXkLMUqRDWbnE zb8tlexvK`=da$6C|92L&|1rfB3dZ?Y`>*KAeRhxk1I*+~kD6!15BVN>Ii7z_<_X3W zZr6XRAx*Yw$i27fp|8RVkFg3RfT+yDW~>e1hs9?wnYC{vwIOf%&y(YoD46AfPX+&eB9QwhidyJGHL_j} zX~XZJ>-oK}#ua!5FCoU=_GKJD(jc_~4`QG4^`!(T7AGpSZgK5{sP{tak#y8BVw1Zh zhT>s-JYc^1O;W%VyPhJ638;le8DVK)Y$6I%y0-j%ACMP{W|LerSVjqQBvzy^f%0bx z)@>BH(mL1>I0kh01Q7k404!5SqxY6)srT(gF;Q-m{!)OdyYRj^m5M|D zC<2g8cQQUNR`r6rBM-XCtFl3)_jLO`N2tA{uoG)^Dt%B=iz2?zA-N4~_E@*6QBNbu zvcrdxeJE+0<{ivxQL#$gNYrQt-d>po;-P5yVX~i~hq==K0m6R#tNqk{xXl2NLNuUa zq{Dq_>-jNB$(#5cz&*1w+ThRNz4NSENrvstMgq5xkRw@<&(Jq46y$073bxK@1`bTX zTJNP^?Ewp3Zob#>l+_lX{8%jr zit?#~zS(u?&d#I#!w&$jEkcWL^y1US_p7XJ)lgihq_fq`0u`$y3#eVqrlxAEz2jNF z-v?A+NAA6$!~?cIAozldxae zC|1Cf0;~21w5pc46+lTc9@=n|&1DAV^<;n+>+`ru;Kb2)T&usJEFK$>6Ld(>P-$mk zLYv&>W!%|aoR4O2<=ifO(^-YHp)`uJBi=KcPHe*u!6?D9k|IB+vTwfo9f3i4)QV{!K@o)^@9xGp~y(u zg8B$q>ZVP1FNESd*s{Zc9XmOadK*1*0J&BM+xD;58+;uW)bsQ2saJGp&#>aH*HC-> z_Rebd_-!v(@v1PYqsJtYA3IqPOXOR=uUNsqUVbdW)SZ+|%*&v07>YTD8^RbPYI!oo zFn=JZhOv2f6)B=;bIo7?<$@babm4uveY8js~iHfUJTg6q}5$;HQ+t#Rd}RE;}1yMToP!o0%DzP$u59(A^wDdR3+#tg8nbNuuPgFHYwD zP%j>J)iJubl=zMAN35#T?`MgPW1M~%SXpVc@P@CeuctpfS_Uk2QDpO_x^<6rB&ck+ ze0_5)ZDO>vPGgDeB~0Y*WDWgntJ3r25;6nD);Nb zLrge6qU$ylmp*@Fm%Efso_z~Ps@@yw>9BnCyK?;D5M~t3j^*R}*rK*R*%dF|z_}o1 zk7=7U@b>?_lB44kA@UM zKCz=BVh5kX9%`|w7m{^mC=0d7Mj>sjE2G?Zewo2(=z+(RZc*LKltl{NTIQ?5Nq&Xi zA{CeG=}oH`dk^rw)Hm5pDwz<&=6-{EYLvNg>_nl zG0q236<8b|hH4FSuwQ)b7I~ z&R_6-LMuVIUL`vbeZl5~2@2YV-XXZ-8cPWN%_m&TC^h^FdlP(TRG;XGjz=F@RjNGm+jM{a|cZcbdMyv zZ2E(svyMKXxbe8v_CfIjgdrP>8wFo*aGw#Le_8$eN``a)_U4};Hr=^>TXv(bcMq#` z(e!M?=4;=At(TX&@|85$_t7^|)rU{DA?!G}eZNzq-^S5)vF$6}tO0)IQZN`q-GlM2 z6pzi84HV98)s~C(BU9wBtv?64GP0VEeUQ@I{<5XuWD28}VL)pNx`u6nV90L1$Y}D@ zypewUDBRIo35#VHDYk$;7=LpkI!tV}2qt9ft?N(0jr9FJr`z^#>?Yq}F}79UAOt^b zypQA88%4Q+4?AAMPnaL_O7m+HfLPvo=WyVfqgxrYGtm-$R@@eLXDBFg<6}D5SY@Sw zl?$`{@0s!`La1dxtq9qctWC+0n=-;7OB-Gm`5B@VXMs~^6g4yy@BFT8J#&m>RJ zaQuo?zosMiiAOotkz?+b>p8{IJ!6tIO7;Ao{7I`>NS_U+PuDZ2M_q@$x@}*z%a^w* z9>B2%eWd&b+u|*cnCYV%=Sf@cD&IY2t5_jeAv)4^=#61H=SRVh_g&8aIKSLWGtD{O zFMq>l=3tx0%G#jYIgV%LGc~U~R-&86>1nbY=UdwEsN5(CbS`XJCVnPS#`#Qe#v8-& z?CQ*V{PfJzz#n@?W|TvIlskp~kh+n@8u~-?hKily4T<5u-pJqR9R5r8)6jLMfk!Pv z*VzXKdT9net^AUuiR{{QPEk_R*P3H>_ku((%{s*@^GKJ`-MTi9iTtiJtf4-;*3VR_ z*ML^5EAprQ3naZ-~YruVDZF@ z$2Nal0L9`eZDo*f-db#+4TXRd6rvfL$bN`^rzY^b7217#D;FJlQrpmktwjftKw!N> zJff|YgaWn~x!AYI&uy<)Aa;o`iM6fWT83lP;0G*R?9PvtLyZzO>wy0Sjbm+& literal 0 HcmV?d00001 From 1c5982567b3f90e922285900d71b085a6d96c203 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 09:47:26 -0400 Subject: [PATCH 62/64] Bump cloud.google.com/go/bigtable from 1.19.0 to 1.20.0 in /sdks (#28789) Bumps [cloud.google.com/go/bigtable](https://github.com/googleapis/google-cloud-go) from 1.19.0 to 1.20.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/documentai/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/video/v1.19.0...video/v1.20.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigtable dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 869db285534d..9c911948c48f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -24,7 +24,7 @@ go 1.20 require ( cloud.google.com/go/bigquery v1.55.0 - cloud.google.com/go/bigtable v1.19.0 + cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 cloud.google.com/go/pubsub v1.33.0 diff --git a/sdks/go.sum b/sdks/go.sum index 52512ffd424c..f1516bdf4a58 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -15,8 +15,8 @@ cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNF cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigquery v1.55.0 h1:hs44Xxov3XLWQiCx2J8lK5U/ihLqnpm4RVVl5fdtLLI= cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= -cloud.google.com/go/bigtable v1.19.0 h1:wiq9LT0kukfInzvy1joMDijCw/OD1UChpSbORXYn0LI= -cloud.google.com/go/bigtable v1.19.0/go.mod h1:xl5kPa8PTkJjdBxg6qdGH88464nNqmbISHSRU+D2yFE= +cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= +cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= From 25024064ed23b37664f165dd6af8a24c972a3148 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 09:47:55 -0400 Subject: [PATCH 63/64] Bump github.com/aws/aws-sdk-go-v2/config in /sdks (#28790) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.18.42 to 1.18.43. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/config/v1.18.42...config/v1.18.43) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config 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 | 8 ++++---- sdks/go.sum | 12 ++++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9c911948c48f..e17427227eba 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,8 +31,8 @@ require ( cloud.google.com/go/spanner v1.49.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.0 - github.com/aws/aws-sdk-go-v2/config v1.18.42 - github.com/aws/aws-sdk-go-v2/credentials v1.13.40 + github.com/aws/aws-sdk-go-v2/config v1.18.43 + github.com/aws/aws-sdk-go-v2/credentials v1.13.41 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 github.com/aws/smithy-go v1.14.2 @@ -107,9 +107,9 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.14.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.22.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index f1516bdf4a58..71c1c4545c89 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,11 +86,13 @@ github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pf github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.18.42 h1:28jHROB27xZwU0CB88giDSjz7M1Sba3olb5JBGwina8= github.com/aws/aws-sdk-go-v2/config v1.18.42/go.mod h1:4AZM3nMMxwlG+eZlxvBKqwVbkDLlnN2a4UGTL6HjaZI= +github.com/aws/aws-sdk-go-v2/config v1.18.43 h1:IgdUtTRvUDC6eiJBqU6vh7bHFNAEBjQ8S+qJ7zVhDOs= +github.com/aws/aws-sdk-go-v2/config v1.18.43/go.mod h1:NiFev8qlgg8MPzw3fO/EwzMZeZwlJEKGwfpjRPA9Nvw= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.40 h1:s8yOkDh+5b1jUDhMBtngF6zKWLDs84chUk2Vk0c38Og= github.com/aws/aws-sdk-go-v2/credentials v1.13.40/go.mod h1:VtEHVAAqDWASwdOqj/1huyT6uHbs5s8FUHfDQdky/Rs= +github.com/aws/aws-sdk-go-v2/credentials v1.13.41 h1:dgbKq1tamtboYAKSXWbqL0lKO9rmEzEhbZFh9JQW/Bg= +github.com/aws/aws-sdk-go-v2/credentials v1.13.41/go.mod h1:cc3Fn7DkKbJalPtQnudHGZZ8ml9+hwtbc1CJONsYYqk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= @@ -121,13 +123,15 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.14.1 h1:YkNzx1RLS0F5qdf9v1Q8Cuv9NXCL2TkosOxhzlUPV64= github.com/aws/aws-sdk-go-v2/service/sso v1.14.1/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 h1:vuGK1vHNP9zx0PfOrtPumbwR2af0ATQ1Z2H6p75AgRQ= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.0/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 h1:8lKOidPkmSmfUtiTgtdXWgaKItCZ/g75/jEk6Ql6GsA= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.22.0 h1:s4bioTgjSFRwOoyEFzAVCmFmoowBgjTR8gkrF/sQ4wk= github.com/aws/aws-sdk-go-v2/service/sts v1.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.14.2 h1:MJU9hqBGbvWZdApzpvoF2WAIJDbtjK2NDJSiJP7HblQ= github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= From b556aee9177b17ae4568f569e2b337f28adbc508 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 10:30:58 -0400 Subject: [PATCH 64/64] Bump urllib3 from 1.26.16 to 1.26.17 in /sdks/python/container/py38 (#28787) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.16 to 1.26.17. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.16...1.26.17) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/py38/base_image_requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index a47fb6beda82..ed5d35fc6457 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -135,7 +135,7 @@ tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.0.1 uritemplate==4.1.1 -urllib3==1.26.16 +urllib3==1.26.17 websocket-client==1.6.3 wrapt==1.15.0 zstandard==0.21.0